You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by sy...@apache.org on 2016/08/17 18:34:07 UTC

[01/50] [abbrv] hbase git commit: HBASE-16299 Update REST API scanner with ability to do reverse scan (Minwoo Kang)

Repository: hbase
Updated Branches:
  refs/heads/hbase-12439 3abd52bdc -> d5080e82f


HBASE-16299 Update REST API scanner with ability to do reverse scan
(Minwoo Kang)


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

Branch: refs/heads/hbase-12439
Commit: 6fdf3aa216d34f2f6fc1ffe7bfea94b8e4bc3f8a
Parents: 0eaf3ed
Author: Ramkrishna <ra...@intel.com>
Authored: Thu Aug 4 11:17:34 2016 +0530
Committer: Ramkrishna <ra...@intel.com>
Committed: Thu Aug 4 11:17:34 2016 +0530

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/rest/Constants.java |  3 +-
 .../apache/hadoop/hbase/rest/TableResource.java |  2 +
 .../apache/hadoop/hbase/rest/TestTableScan.java | 52 ++++++++++++++++++++
 3 files changed, 56 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/6fdf3aa2/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/Constants.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/Constants.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/Constants.java
index 505dbb3..54ce117 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/Constants.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/Constants.java
@@ -68,7 +68,8 @@ public interface Constants {
   String SCAN_BATCH_SIZE = "batchsize";
   String SCAN_LIMIT = "limit";
   String SCAN_FETCH_SIZE = "hbase.rest.scan.fetchsize";
-  String SCAN_FILTER = "filter"; 
+  String SCAN_FILTER = "filter";
+  String SCAN_REVERSED = "reversed";
   String CUSTOM_FILTERS = "hbase.rest.custom.filters"; 
 
   String ROW_KEYS_PARAM_NAME = "row";

http://git-wip-us.apache.org/repos/asf/hbase/blob/6fdf3aa2/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/TableResource.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/TableResource.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/TableResource.java
index 45e7caf..9eb21ec 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/TableResource.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/TableResource.java
@@ -134,6 +134,7 @@ public class TableResource extends ResourceBase {
       @DefaultValue("0") @QueryParam(Constants.SCAN_START_TIME) long startTime,
       @DefaultValue(Long.MAX_VALUE + "") @QueryParam(Constants.SCAN_END_TIME) long endTime,
       @DefaultValue("true") @QueryParam(Constants.SCAN_BATCH_SIZE) boolean cacheBlocks,
+      @DefaultValue("false") @QueryParam(Constants.SCAN_REVERSED) boolean reversed,
       @DefaultValue("") @QueryParam(Constants.SCAN_FILTER) String filters) {
     try {
       Filter filter = null;
@@ -200,6 +201,7 @@ public class TableResource extends ResourceBase {
       }
       int fetchSize = this.servlet.getConfiguration().getInt(Constants.SCAN_FETCH_SIZE, 10);
       tableScan.setCaching(fetchSize);
+      tableScan.setReversed(reversed);
       return new TableScanResource(hTable.getScanner(tableScan), userRequestedLimit);
     } catch (IOException exp) {
       servlet.getMetrics().incrementFailedScanRequests(1);

http://git-wip-us.apache.org/repos/asf/hbase/blob/6fdf3aa2/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestTableScan.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestTableScan.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestTableScan.java
index 789e9e1..7924aed 100644
--- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestTableScan.java
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestTableScan.java
@@ -29,6 +29,8 @@ import java.io.InputStream;
 import java.io.Serializable;
 import java.net.URLEncoder;
 import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
 import java.util.List;
 
 import javax.ws.rs.core.MediaType;
@@ -548,6 +550,56 @@ public class TestTableScan {
     assertEquals(0, count);
   }
 
+  @Test
+  public void testReversed() throws IOException, JAXBException {
+    StringBuilder builder = new StringBuilder();
+    builder.append("/*");
+    builder.append("?");
+    builder.append(Constants.SCAN_COLUMN + "=" + COLUMN_1);
+    builder.append("&");
+    builder.append(Constants.SCAN_START_ROW + "=aaa");
+    builder.append("&");
+    builder.append(Constants.SCAN_END_ROW + "=aay");
+    Response response = client.get("/" + TABLE + builder.toString(), Constants.MIMETYPE_XML);
+    assertEquals(200, response.getCode());
+    JAXBContext ctx = JAXBContext.newInstance(CellSetModel.class);
+    Unmarshaller ush = ctx.createUnmarshaller();
+    CellSetModel model = (CellSetModel) ush.unmarshal(response.getStream());
+    int count = TestScannerResource.countCellSet(model);
+    assertEquals(24, count);
+    List<RowModel> rowModels = model.getRows().subList(1, count);
+
+    //reversed
+    builder = new StringBuilder();
+    builder.append("/*");
+    builder.append("?");
+    builder.append(Constants.SCAN_COLUMN + "=" + COLUMN_1);
+    builder.append("&");
+    builder.append(Constants.SCAN_START_ROW + "=aay");
+    builder.append("&");
+    builder.append(Constants.SCAN_END_ROW + "=aaa");
+    builder.append("&");
+    builder.append(Constants.SCAN_REVERSED + "=true");
+    response = client.get("/" + TABLE + builder.toString(), Constants.MIMETYPE_XML);
+    assertEquals(200, response.getCode());
+    model = (CellSetModel) ush.unmarshal(response.getStream());
+    count = TestScannerResource.countCellSet(model);
+    assertEquals(24, count);
+    List<RowModel> reversedRowModels = model.getRows().subList(1, count);
+
+    Collections.reverse(reversedRowModels);
+    assertEquals(rowModels.size(), reversedRowModels.size());
+    for (int i = 0; i < rowModels.size(); i++) {
+      RowModel rowModel = rowModels.get(i);
+      RowModel reversedRowModel = reversedRowModels.get(i);
+
+      assertEquals(new String(rowModel.getKey(), "UTF-8"),
+          new String(reversedRowModel.getKey(), "UTF-8"));
+      assertEquals(new String(rowModel.getCells().get(0).getValue(), "UTF-8"),
+          new String(reversedRowModel.getCells().get(0).getValue(), "UTF-8"));
+    }
+  }
+
   public static class CustomFilter extends PrefixFilter {
     private byte[] key = null;
 


[39/50] [abbrv] hbase git commit: REVERT of revert of "HBASE-16308 Contain protobuf references Gather up the pb references into a few locations only rather than have pb references distributed all about the code base." This is a revert of a revert; i.e. w

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/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 29650ef..48a614f 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
@@ -18,10 +18,6 @@
  */
 package org.apache.hadoop.hbase.client;
 
-import com.google.common.annotations.VisibleForTesting;
-import com.google.protobuf.ByteString;
-import com.google.protobuf.ServiceException;
-
 import java.io.Closeable;
 import java.io.IOException;
 import java.io.InterruptedIOException;
@@ -32,6 +28,7 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
@@ -69,7 +66,6 @@ import org.apache.hadoop.hbase.ZooKeeperConnectionException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.client.security.SecurityCapability;
-import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
 import org.apache.hadoop.hbase.ipc.MasterCoprocessorRpcChannel;
@@ -183,6 +179,9 @@ import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.zookeeper.KeeperException;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.ServiceException;
+
 /**
  * HBaseAdmin is no longer a client API. It is marked InterfaceAudience.Private indicating that
  * this is an HBase-internal class as defined in
@@ -211,10 +210,6 @@ public class HBaseAdmin implements Admin {
   private volatile Configuration conf;
   private final long pause;
   private final int numRetries;
-  // Some operations can take a long time such as disable of big table.
-  // numRetries is for 'normal' stuff... Multiply by this factor when
-  // want to wait a long time.
-  private final int retryLongerMultiplier;
   private final int syncWaitTimeout;
   private boolean aborted;
   private int operationTimeout;
@@ -239,8 +234,6 @@ public class HBaseAdmin implements Admin {
         HConstants.DEFAULT_HBASE_CLIENT_PAUSE);
     this.numRetries = this.conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
         HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
-    this.retryLongerMultiplier = this.conf.getInt(
-        "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,
@@ -262,7 +255,7 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public boolean isAborted(){
+  public boolean isAborted() {
     return this.aborted;
   }
 
@@ -274,24 +267,19 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public Future<Boolean> abortProcedureAsync(
-      final long procId,
-      final boolean mayInterruptIfRunning) throws IOException {
-    Boolean abortProcResponse = executeCallable(
-      new MasterCallable<AbortProcedureResponse>(getConnection()) {
-        @Override
-        public AbortProcedureResponse call(int callTimeout) throws ServiceException {
-          PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-          controller.setCallTimeout(callTimeout);
-          AbortProcedureRequest abortProcRequest =
-              AbortProcedureRequest.newBuilder().setProcId(procId).build();
-          return master.abortProcedure(controller, abortProcRequest);
-        }
-      }).getIsProcedureAborted();
-
-    AbortProcedureFuture abortProcFuture =
-        new AbortProcedureFuture(this, procId, abortProcResponse);
-    return abortProcFuture;
+  public Future<Boolean> abortProcedureAsync(final long procId, final boolean mayInterruptIfRunning)
+      throws IOException {
+    Boolean abortProcResponse =
+        executeCallable(new MasterCallable<AbortProcedureResponse>(getConnection(),
+            getRpcControllerFactory()) {
+      @Override
+      protected AbortProcedureResponse rpcCall() throws Exception {
+        AbortProcedureRequest abortProcRequest =
+            AbortProcedureRequest.newBuilder().setProcId(procId).build();
+        return master.abortProcedure(getRpcController(), abortProcRequest);
+      }
+    }).getIsProcedureAborted();
+    return new AbortProcedureFuture(this, procId, abortProcResponse);
   }
 
   private static class AbortProcedureFuture extends ProcedureFuture<Boolean> {
@@ -324,9 +312,9 @@ public class HBaseAdmin implements Admin {
 
   @Override
   public boolean tableExists(final TableName tableName) throws IOException {
-    return executeCallable(new ConnectionCallable<Boolean>(getConnection()) {
+    return executeCallable(new RpcRetryingCallable<Boolean>() {
       @Override
-      public Boolean call(int callTimeout) throws ServiceException, IOException {
+      protected Boolean rpcCall(int callTimeout) throws Exception {
         return MetaTableAccessor.tableExists(connection, tableName);
       }
     });
@@ -350,14 +338,14 @@ public class HBaseAdmin implements Admin {
   @Override
   public HTableDescriptor[] listTables(final Pattern pattern, final boolean includeSysTables)
       throws IOException {
-    return executeCallable(new MasterCallable<HTableDescriptor[]>(getConnection()) {
+    return executeCallable(new MasterCallable<HTableDescriptor[]>(getConnection(),
+        getRpcControllerFactory()) {
       @Override
-      public HTableDescriptor[] call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
+      protected HTableDescriptor[] rpcCall() throws Exception {
         GetTableDescriptorsRequest req =
             RequestConverter.buildGetTableDescriptorsRequest(pattern, includeSysTables);
-        return ProtobufUtil.getHTableDescriptorArray(master.getTableDescriptors(controller, req));
+        return ProtobufUtil.getHTableDescriptorArray(master.getTableDescriptors(getRpcController(),
+            req));
       }
     });
   }
@@ -386,14 +374,13 @@ public class HBaseAdmin implements Admin {
   @Override
   public TableName[] listTableNames(final Pattern pattern, final boolean includeSysTables)
       throws IOException {
-    return executeCallable(new MasterCallable<TableName[]>(getConnection()) {
+    return executeCallable(new MasterCallable<TableName[]>(getConnection(),
+        getRpcControllerFactory()) {
       @Override
-      public TableName[] call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
+      protected TableName[] rpcCall() throws Exception {
         GetTableNamesRequest req =
             RequestConverter.buildGetTableNamesRequest(pattern, includeSysTables);
-        return ProtobufUtil.getTableNameArray(master.getTableNames(controller, req)
+        return ProtobufUtil.getTableNameArray(master.getTableNames(getRpcController(), req)
             .getTableNamesList());
       }
     });
@@ -414,27 +401,24 @@ public class HBaseAdmin implements Admin {
   static HTableDescriptor getTableDescriptor(final TableName tableName, Connection connection,
       RpcRetryingCallerFactory rpcCallerFactory, final RpcControllerFactory rpcControllerFactory,
       int operationTimeout, int rpcTimeout) throws IOException {
-      if (tableName == null) return null;
-      HTableDescriptor htd = executeCallable(new MasterCallable<HTableDescriptor>(connection) {
-        @Override
-        public HTableDescriptor call(int callTimeout) throws ServiceException {
-          PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-          controller.setCallTimeout(callTimeout);
-          GetTableDescriptorsResponse htds;
-          GetTableDescriptorsRequest req =
-                  RequestConverter.buildGetTableDescriptorsRequest(tableName);
-          htds = master.getTableDescriptors(controller, req);
-
-          if (!htds.getTableSchemaList().isEmpty()) {
-            return ProtobufUtil.convertToHTableDesc(htds.getTableSchemaList().get(0));
-          }
-          return null;
+    if (tableName == null) return null;
+    HTableDescriptor htd =
+        executeCallable(new MasterCallable<HTableDescriptor>(connection, rpcControllerFactory) {
+      @Override
+      protected HTableDescriptor rpcCall() throws Exception {
+        GetTableDescriptorsRequest req =
+            RequestConverter.buildGetTableDescriptorsRequest(tableName);
+        GetTableDescriptorsResponse htds = master.getTableDescriptors(getRpcController(), req);
+        if (!htds.getTableSchemaList().isEmpty()) {
+          return ProtobufUtil.convertToHTableDesc(htds.getTableSchemaList().get(0));
         }
-      }, rpcCallerFactory, operationTimeout, rpcTimeout);
-      if (htd != null) {
-        return htd;
+        return null;
       }
-      throw new TableNotFoundException(tableName.getNameAsString());
+    }, rpcCallerFactory, operationTimeout, rpcTimeout);
+    if (htd != null) {
+      return htd;
+    }
+    throw new TableNotFoundException(tableName.getNameAsString());
   }
 
   private long getPauseTime(int tries) {
@@ -502,15 +486,13 @@ public class HBaseAdmin implements Admin {
     }
 
     CreateTableResponse response = executeCallable(
-      new MasterCallable<CreateTableResponse>(getConnection()) {
+      new MasterCallable<CreateTableResponse>(getConnection(), getRpcControllerFactory()) {
         @Override
-        public CreateTableResponse call(int callTimeout) throws ServiceException {
-          PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-          controller.setCallTimeout(callTimeout);
-          controller.setPriority(desc.getTableName());
+        protected CreateTableResponse rpcCall() throws Exception {
+          setPriority(desc.getTableName());
           CreateTableRequest request = RequestConverter.buildCreateTableRequest(
             desc, splitKeys, ng.getNonceGroup(), ng.newNonce());
-          return master.createTable(controller, request);
+          return master.createTable(getRpcController(), request);
         }
       });
     return new CreateTableFuture(this, desc, splitKeys, response);
@@ -554,15 +536,13 @@ public class HBaseAdmin implements Admin {
   @Override
   public Future<Void> deleteTableAsync(final TableName tableName) throws IOException {
     DeleteTableResponse response = executeCallable(
-      new MasterCallable<DeleteTableResponse>(getConnection()) {
+      new MasterCallable<DeleteTableResponse>(getConnection(), getRpcControllerFactory()) {
         @Override
-        public DeleteTableResponse call(int callTimeout) throws ServiceException {
-          PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-          controller.setCallTimeout(callTimeout);
-          controller.setPriority(tableName);
+        protected DeleteTableResponse rpcCall() throws Exception {
+          setPriority(tableName);
           DeleteTableRequest req =
               RequestConverter.buildDeleteTableRequest(tableName, ng.getNonceGroup(),ng.newNonce());
-          return master.deleteTable(controller,req);
+          return master.deleteTable(getRpcController(), req);
         }
       });
     return new DeleteTableFuture(this, tableName, response);
@@ -636,16 +616,15 @@ public class HBaseAdmin implements Admin {
   public Future<Void> truncateTableAsync(final TableName tableName, final boolean preserveSplits)
       throws IOException {
     TruncateTableResponse response =
-        executeCallable(new MasterCallable<TruncateTableResponse>(getConnection()) {
+        executeCallable(new MasterCallable<TruncateTableResponse>(getConnection(),
+            getRpcControllerFactory()) {
           @Override
-          public TruncateTableResponse call(int callTimeout) throws ServiceException {
-            PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-            controller.setCallTimeout(callTimeout);
-            controller.setPriority(tableName);
+          protected TruncateTableResponse rpcCall() throws Exception {
+            setPriority(tableName);
             LOG.info("Started truncating " + tableName);
             TruncateTableRequest req = RequestConverter.buildTruncateTableRequest(
               tableName, preserveSplits, ng.getNonceGroup(), ng.newNonce());
-            return master.truncateTable(controller, req);
+            return master.truncateTable(getRpcController(), req);
           }
         });
     return new TruncateTableFuture(this, tableName, preserveSplits, response);
@@ -701,17 +680,14 @@ public class HBaseAdmin implements Admin {
   public Future<Void> enableTableAsync(final TableName tableName) throws IOException {
     TableName.isLegalFullyQualifiedTableName(tableName.getName());
     EnableTableResponse response = executeCallable(
-      new MasterCallable<EnableTableResponse>(getConnection()) {
+      new MasterCallable<EnableTableResponse>(getConnection(), getRpcControllerFactory()) {
         @Override
-        public EnableTableResponse call(int callTimeout) throws ServiceException {
-          PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-          controller.setCallTimeout(callTimeout);
-          controller.setPriority(tableName);
-
+        protected EnableTableResponse rpcCall() throws Exception {
+          setPriority(tableName);
           LOG.info("Started enable of " + tableName);
           EnableTableRequest req =
               RequestConverter.buildEnableTableRequest(tableName, ng.getNonceGroup(),ng.newNonce());
-          return master.enableTable(controller,req);
+          return master.enableTable(getRpcController(),req);
         }
       });
     return new EnableTableFuture(this, tableName, response);
@@ -767,18 +743,15 @@ public class HBaseAdmin implements Admin {
   public Future<Void> disableTableAsync(final TableName tableName) throws IOException {
     TableName.isLegalFullyQualifiedTableName(tableName.getName());
     DisableTableResponse response = executeCallable(
-      new MasterCallable<DisableTableResponse>(getConnection()) {
+      new MasterCallable<DisableTableResponse>(getConnection(), getRpcControllerFactory()) {
         @Override
-        public DisableTableResponse call(int callTimeout) throws ServiceException {
-          PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-          controller.setCallTimeout(callTimeout);
-          controller.setPriority(tableName);
-
+        protected DisableTableResponse rpcCall() throws Exception {
+          setPriority(tableName);
           LOG.info("Started disable of " + tableName);
           DisableTableRequest req =
               RequestConverter.buildDisableTableRequest(
                 tableName, ng.getNonceGroup(), ng.newNonce());
-          return master.disableTable(controller, req);
+          return master.disableTable(getRpcController(), req);
         }
       });
     return new DisableTableFuture(this, tableName, response);
@@ -827,12 +800,13 @@ public class HBaseAdmin implements Admin {
   @Override
   public boolean isTableEnabled(final TableName tableName) throws IOException {
     checkTableExists(tableName);
-    return executeCallable(new ConnectionCallable<Boolean>(getConnection()) {
+    return executeCallable(new RpcRetryingCallable<Boolean>() {
       @Override
-      public Boolean call(int callTimeout) throws ServiceException, IOException {
-        TableState tableState = MetaTableAccessor.getTableState(connection, tableName);
-        if (tableState == null)
+      protected Boolean rpcCall(int callTimeout) throws Exception {
+        TableState tableState = MetaTableAccessor.getTableState(getConnection(), tableName);
+        if (tableState == null) {
           throw new TableNotFoundException(tableName);
+        }
         return tableState.inStates(TableState.State.ENABLED);
       }
     });
@@ -856,16 +830,14 @@ public class HBaseAdmin implements Admin {
 
   @Override
   public Pair<Integer, Integer> getAlterStatus(final TableName tableName) throws IOException {
-    return executeCallable(new MasterCallable<Pair<Integer, Integer>>(getConnection()) {
+    return executeCallable(new MasterCallable<Pair<Integer, Integer>>(getConnection(),
+        getRpcControllerFactory()) {
       @Override
-      public Pair<Integer, Integer> call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
-        controller.setPriority(tableName);
-
+      protected Pair<Integer, Integer> rpcCall() throws Exception {
+        setPriority(tableName);
         GetSchemaAlterStatusRequest req = RequestConverter
             .buildGetSchemaAlterStatusRequest(tableName);
-        GetSchemaAlterStatusResponse ret = master.getSchemaAlterStatus(controller, req);
+        GetSchemaAlterStatusResponse ret = master.getSchemaAlterStatus(getRpcController(), req);
         Pair<Integer, Integer> pair = new Pair<>(ret.getYetToUpdateRegions(),
             ret.getTotalRegions());
         return pair;
@@ -894,17 +866,15 @@ public class HBaseAdmin implements Admin {
   public Future<Void> addColumnFamily(final TableName tableName,
       final HColumnDescriptor columnFamily) throws IOException {
     AddColumnResponse response =
-        executeCallable(new MasterCallable<AddColumnResponse>(getConnection()) {
+        executeCallable(new MasterCallable<AddColumnResponse>(getConnection(),
+            getRpcControllerFactory()) {
           @Override
-          public AddColumnResponse call(int callTimeout) throws ServiceException {
-            PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-            controller.setCallTimeout(callTimeout);
-            controller.setPriority(tableName);
-
+          protected AddColumnResponse rpcCall() throws Exception {
+            setPriority(tableName);
             AddColumnRequest req =
                 RequestConverter.buildAddColumnRequest(tableName, columnFamily, ng.getNonceGroup(),
                   ng.newNonce());
-            return master.addColumn(controller, req);
+            return master.addColumn(getRpcController(), req);
           }
         });
     return new AddColumnFamilyFuture(this, tableName, response);
@@ -939,17 +909,15 @@ public class HBaseAdmin implements Admin {
   public Future<Void> deleteColumnFamily(final TableName tableName, final byte[] columnFamily)
       throws IOException {
     DeleteColumnResponse response =
-        executeCallable(new MasterCallable<DeleteColumnResponse>(getConnection()) {
+        executeCallable(new MasterCallable<DeleteColumnResponse>(getConnection(),
+            getRpcControllerFactory()) {
           @Override
-          public DeleteColumnResponse call(int callTimeout) throws ServiceException {
-            PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-            controller.setCallTimeout(callTimeout);
-            controller.setPriority(tableName);
-
+          protected DeleteColumnResponse rpcCall() throws Exception {
+            setPriority(tableName);
             DeleteColumnRequest req =
                 RequestConverter.buildDeleteColumnRequest(tableName, columnFamily,
                   ng.getNonceGroup(), ng.newNonce());
-            master.deleteColumn(controller, req);
+            master.deleteColumn(getRpcController(), req);
             return null;
           }
         });
@@ -985,17 +953,15 @@ public class HBaseAdmin implements Admin {
   public Future<Void> modifyColumnFamily(final TableName tableName,
       final HColumnDescriptor columnFamily) throws IOException {
     ModifyColumnResponse response =
-        executeCallable(new MasterCallable<ModifyColumnResponse>(getConnection()) {
+        executeCallable(new MasterCallable<ModifyColumnResponse>(getConnection(),
+            getRpcControllerFactory()) {
           @Override
-          public ModifyColumnResponse call(int callTimeout) throws ServiceException {
-            PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-            controller.setCallTimeout(callTimeout);
-            controller.setPriority(tableName);
-
+          protected ModifyColumnResponse rpcCall() throws Exception {
+            setPriority(tableName);
             ModifyColumnRequest req =
                 RequestConverter.buildModifyColumnRequest(tableName, columnFamily,
                   ng.getNonceGroup(), ng.newNonce());
-            master.modifyColumn(controller, req);
+            master.modifyColumn(getRpcController(), req);
             return null;
           }
         });
@@ -1043,34 +1009,34 @@ public class HBaseAdmin implements Admin {
 
   @Override
   public boolean closeRegionWithEncodedRegionName(final String encodedRegionName,
-      final String serverName) throws IOException {
+      final String serverName)
+  throws IOException {
     if (null == serverName || ("").equals(serverName.trim())) {
-      throw new IllegalArgumentException(
-          "The servername cannot be null or empty.");
+      throw new IllegalArgumentException("The servername cannot be null or empty.");
     }
     ServerName sn = ServerName.valueOf(serverName);
     AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
     // Close the region without updating zk state.
     CloseRegionRequest request =
       RequestConverter.buildCloseRegionRequest(sn, encodedRegionName);
+    // TODO: There is no timeout on this controller. Set one!
+    PayloadCarryingRpcController controller = this.rpcControllerFactory.newController();
     try {
-      PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-
-      // TODO: this does not do retries, it should. Set priority and timeout in controller
       CloseRegionResponse response = admin.closeRegion(controller, request);
-      boolean isRegionClosed = response.getClosed();
-      if (false == isRegionClosed) {
+      boolean closed = response.getClosed();
+      if (false == closed) {
         LOG.error("Not able to close the region " + encodedRegionName + ".");
       }
-      return isRegionClosed;
-    } catch (ServiceException se) {
-      throw ProtobufUtil.getRemoteException(se);
+      return closed;
+    } catch (Exception e) {
+      throw ProtobufUtil.handleRemoteException(e);
     }
   }
 
   @Override
   public void closeRegion(final ServerName sn, final HRegionInfo hri) throws IOException {
     AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
+    // TODO: There is no timeout on this controller. Set one!
     PayloadCarryingRpcController controller = rpcControllerFactory.newController();
 
     // Close the region without updating zk state.
@@ -1080,6 +1046,7 @@ public class HBaseAdmin implements Admin {
   @Override
   public List<HRegionInfo> getOnlineRegions(final ServerName sn) throws IOException {
     AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
+    // TODO: There is no timeout on this controller. Set one!
     PayloadCarryingRpcController controller = rpcControllerFactory.newController();
     return ProtobufUtil.getOnlineRegions(controller, admin);
   }
@@ -1104,20 +1071,21 @@ public class HBaseAdmin implements Admin {
     if (regionServerPair.getSecond() == null) {
       throw new NoServerForRegionException(Bytes.toStringBinary(regionName));
     }
-    HRegionInfo hRegionInfo = regionServerPair.getFirst();
+    final HRegionInfo hRegionInfo = regionServerPair.getFirst();
     ServerName serverName = regionServerPair.getSecond();
-
-    PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-
-    AdminService.BlockingInterface admin = this.connection.getAdmin(serverName);
-    FlushRegionRequest request =
-        RequestConverter.buildFlushRegionRequest(hRegionInfo.getRegionName());
-    try {
-      // TODO: this does not do retries, it should. Set priority and timeout in controller
-      admin.flushRegion(controller, request);
-    } catch (ServiceException se) {
-      throw ProtobufUtil.getRemoteException(se);
-    }
+    final AdminService.BlockingInterface admin = this.connection.getAdmin(serverName);
+    Callable<Void> callable = new Callable<Void>() {
+      @Override
+      public Void call() throws Exception {
+        // TODO: There is no timeout on this controller. Set one!
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        FlushRegionRequest request =
+            RequestConverter.buildFlushRegionRequest(hRegionInfo.getRegionName());
+        admin.flushRegion(controller, request);
+        return null;
+      }
+    };
+    ProtobufUtil.call(callable);
   }
 
   /**
@@ -1268,67 +1236,46 @@ public class HBaseAdmin implements Admin {
   private void compact(final ServerName sn, final HRegionInfo hri,
       final boolean major, final byte [] family)
   throws IOException {
-    PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-    AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
-    CompactRegionRequest request =
-      RequestConverter.buildCompactRegionRequest(hri.getRegionName(), major, family);
-    try {
-      // TODO: this does not do retries, it should. Set priority and timeout in controller
-      admin.compactRegion(controller, request);
-    } catch (ServiceException se) {
-      throw ProtobufUtil.getRemoteException(se);
-    }
+    final AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
+    Callable<Void> callable = new Callable<Void>() {
+      @Override
+      public Void call() throws Exception {
+        // TODO: There is no timeout on this controller. Set one!
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        CompactRegionRequest request =
+            RequestConverter.buildCompactRegionRequest(hri.getRegionName(), major, family);
+        admin.compactRegion(controller, request);
+        return null;
+      }
+    };
+    ProtobufUtil.call(callable);
   }
 
   @Override
   public void move(final byte [] encodedRegionName, final byte [] destServerName)
-      throws IOException {
-
-    executeCallable(new MasterCallable<Void>(getConnection()) {
+  throws IOException {
+    executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
       @Override
-      public Void call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
-        // Hard to know the table name, at least check if meta
-        if (isMetaRegion(encodedRegionName)) {
-          controller.setPriority(TableName.META_TABLE_NAME);
-        }
-
-        try {
-          MoveRegionRequest request =
-              RequestConverter.buildMoveRegionRequest(encodedRegionName, destServerName);
-            master.moveRegion(controller, request);
-        } catch (DeserializationException de) {
-          LOG.error("Could not parse destination server name: " + de);
-          throw new ServiceException(new DoNotRetryIOException(de));
-        }
+      protected Void rpcCall() throws Exception {
+        setPriority(encodedRegionName);
+        MoveRegionRequest request =
+            RequestConverter.buildMoveRegionRequest(encodedRegionName, destServerName);
+        master.moveRegion(getRpcController(), request);
         return null;
       }
     });
   }
 
-  private boolean isMetaRegion(final byte[] regionName) {
-    return Bytes.equals(regionName, HRegionInfo.FIRST_META_REGIONINFO.getRegionName())
-        || Bytes.equals(regionName, HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes());
-  }
-
   @Override
-  public void assign(final byte[] regionName) throws MasterNotRunningException,
+  public void assign(final byte [] regionName) throws MasterNotRunningException,
       ZooKeeperConnectionException, IOException {
-    final byte[] toBeAssigned = getRegionName(regionName);
-    executeCallable(new MasterCallable<Void>(getConnection()) {
+    executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
       @Override
-      public Void call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
-        // Hard to know the table name, at least check if meta
-        if (isMetaRegion(regionName)) {
-          controller.setPriority(TableName.META_TABLE_NAME);
-        }
-
+      protected Void rpcCall() throws Exception {
+        setPriority(regionName);
         AssignRegionRequest request =
-          RequestConverter.buildAssignRegionRequest(toBeAssigned);
-        master.assignRegion(controller,request);
+            RequestConverter.buildAssignRegionRequest(getRegionName(regionName));
+        master.assignRegion(getRpcController(), request);
         return null;
       }
     });
@@ -1338,18 +1285,13 @@ public class HBaseAdmin implements Admin {
   public void unassign(final byte [] regionName, final boolean force)
   throws MasterNotRunningException, ZooKeeperConnectionException, IOException {
     final byte[] toBeUnassigned = getRegionName(regionName);
-    executeCallable(new MasterCallable<Void>(getConnection()) {
+    executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
       @Override
-      public Void call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
-        // Hard to know the table name, at least check if meta
-        if (isMetaRegion(regionName)) {
-          controller.setPriority(TableName.META_TABLE_NAME);
-        }
+      protected Void rpcCall() throws Exception {
+        setPriority(regionName);
         UnassignRegionRequest request =
-          RequestConverter.buildUnassignRegionRequest(toBeUnassigned, force);
-        master.unassignRegion(controller, request);
+            RequestConverter.buildUnassignRegionRequest(toBeUnassigned, force);
+        master.unassignRegion(getRpcController(), request);
         return null;
       }
     });
@@ -1358,16 +1300,12 @@ public class HBaseAdmin implements Admin {
   @Override
   public void offline(final byte [] regionName)
   throws IOException {
-    executeCallable(new MasterCallable<Void>(getConnection()) {
+    executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
       @Override
-      public Void call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
-        // Hard to know the table name, at least check if meta
-        if (isMetaRegion(regionName)) {
-          controller.setPriority(TableName.META_TABLE_NAME);
-        }
-        master.offlineRegion(controller, RequestConverter.buildOfflineRegionRequest(regionName));
+      protected Void rpcCall() throws Exception {
+        setPriority(regionName);
+        master.offlineRegion(getRpcController(),
+            RequestConverter.buildOfflineRegionRequest(regionName));
         return null;
       }
     });
@@ -1376,56 +1314,44 @@ public class HBaseAdmin implements Admin {
   @Override
   public boolean setBalancerRunning(final boolean on, final boolean synchronous)
   throws IOException {
-    return executeCallable(new MasterCallable<Boolean>(getConnection()) {
+    return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
       @Override
-      public Boolean call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
-
+      protected Boolean rpcCall() throws Exception {
         SetBalancerRunningRequest req =
             RequestConverter.buildSetBalancerRunningRequest(on, synchronous);
-        return master.setBalancerRunning(controller, req).getPrevBalanceValue();
+        return master.setBalancerRunning(getRpcController(), req).getPrevBalanceValue();
       }
     });
   }
 
   @Override
   public boolean balancer() throws IOException {
-    return executeCallable(new MasterCallable<Boolean>(getConnection()) {
+    return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
       @Override
-      public Boolean call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
-
-        return master.balance(controller,
-          RequestConverter.buildBalanceRequest(false)).getBalancerRan();
+      protected Boolean rpcCall() throws Exception {
+        return master.balance(getRpcController(),
+            RequestConverter.buildBalanceRequest(false)).getBalancerRan();
       }
     });
   }
 
   @Override
   public boolean balancer(final boolean force) throws IOException {
-    return executeCallable(new MasterCallable<Boolean>(getConnection()) {
+    return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
       @Override
-      public Boolean call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
-
-        return master.balance(controller,
-          RequestConverter.buildBalanceRequest(force)).getBalancerRan();
+      protected Boolean rpcCall() throws Exception {
+        return master.balance(getRpcController(),
+            RequestConverter.buildBalanceRequest(force)).getBalancerRan();
       }
     });
   }
 
   @Override
   public boolean isBalancerEnabled() throws IOException {
-    return executeCallable(new MasterCallable<Boolean>(getConnection()) {
+    return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
       @Override
-      public Boolean call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
-
-        return master.isBalancerEnabled(controller,
+      protected Boolean rpcCall() throws Exception {
+        return master.isBalancerEnabled(getRpcController(),
           RequestConverter.buildIsBalancerEnabledRequest()).getEnabled();
       }
     });
@@ -1433,27 +1359,21 @@ public class HBaseAdmin implements Admin {
 
   @Override
   public boolean normalize() throws IOException {
-    return executeCallable(new MasterCallable<Boolean>(getConnection()) {
+    return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
       @Override
-      public Boolean call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
-
-        return master.normalize(controller,
-          RequestConverter.buildNormalizeRequest()).getNormalizerRan();
+      protected Boolean rpcCall() throws Exception {
+        return master.normalize(getRpcController(),
+            RequestConverter.buildNormalizeRequest()).getNormalizerRan();
       }
     });
   }
 
   @Override
   public boolean isNormalizerEnabled() throws IOException {
-    return executeCallable(new MasterCallable<Boolean>(getConnection()) {
+    return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
       @Override
-      public Boolean call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
-
-        return master.isNormalizerEnabled(controller,
+      protected Boolean rpcCall() throws Exception {
+        return master.isNormalizerEnabled(getRpcController(),
           RequestConverter.buildIsNormalizerEnabledRequest()).getEnabled();
       }
     });
@@ -1461,28 +1381,22 @@ public class HBaseAdmin implements Admin {
 
   @Override
   public boolean setNormalizerRunning(final boolean on) throws IOException {
-    return executeCallable(new MasterCallable<Boolean>(getConnection()) {
+    return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
       @Override
-      public Boolean call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
-
+      protected Boolean rpcCall() throws Exception {
         SetNormalizerRunningRequest req =
           RequestConverter.buildSetNormalizerRunningRequest(on);
-        return master.setNormalizerRunning(controller, req).getPrevNormalizerValue();
+        return master.setNormalizerRunning(getRpcController(), req).getPrevNormalizerValue();
       }
     });
   }
 
   @Override
   public boolean enableCatalogJanitor(final boolean enable) throws IOException {
-    return executeCallable(new MasterCallable<Boolean>(getConnection()) {
+    return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
       @Override
-      public Boolean call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
-
-        return master.enableCatalogJanitor(controller,
+      protected Boolean rpcCall() throws Exception {
+        return master.enableCatalogJanitor(getRpcController(),
           RequestConverter.buildEnableCatalogJanitorRequest(enable)).getPrevValue();
       }
     });
@@ -1490,13 +1404,10 @@ public class HBaseAdmin implements Admin {
 
   @Override
   public int runCatalogScan() throws IOException {
-    return executeCallable(new MasterCallable<Integer>(getConnection()) {
+    return executeCallable(new MasterCallable<Integer>(getConnection(), getRpcControllerFactory()) {
       @Override
-      public Integer call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
-
-        return master.runCatalogScan(controller,
+      protected Integer rpcCall() throws Exception {
+        return master.runCatalogScan(getRpcController(),
           RequestConverter.buildCatalogScanRequest()).getScanResult();
       }
     });
@@ -1504,13 +1415,10 @@ public class HBaseAdmin implements Admin {
 
   @Override
   public boolean isCatalogJanitorEnabled() throws IOException {
-    return executeCallable(new MasterCallable<Boolean>(getConnection()) {
+    return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
       @Override
-      public Boolean call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
-
-        return master.isCatalogJanitorEnabled(controller,
+      protected Boolean rpcCall() throws Exception {
+        return master.isCatalogJanitorEnabled(getRpcController(),
           RequestConverter.buildIsCatalogJanitorEnabledRequest()).getValue();
       }
     });
@@ -1616,25 +1524,18 @@ public class HBaseAdmin implements Admin {
     }
 
     DispatchMergingRegionsResponse response =
-      executeCallable(new MasterCallable<DispatchMergingRegionsResponse>(getConnection()) {
+        executeCallable(new MasterCallable<DispatchMergingRegionsResponse>(getConnection(),
+            getRpcControllerFactory()) {
       @Override
-      public DispatchMergingRegionsResponse call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
-
-        try {
-          DispatchMergingRegionsRequest request = RequestConverter
-              .buildDispatchMergingRegionsRequest(
+      protected DispatchMergingRegionsResponse rpcCall() throws Exception {
+        DispatchMergingRegionsRequest request = RequestConverter
+            .buildDispatchMergingRegionsRequest(
                 encodedNameOfRegionA,
                 encodedNameOfRegionB,
                 forcible,
                 ng.getNonceGroup(),
                 ng.newNonce());
-          return master.dispatchMergingRegions(controller, request);
-        } catch (DeserializationException de) {
-          LOG.error("Could not parse destination server name: " + de);
-          throw new ServiceException(new DoNotRetryIOException(de));
-        }
+        return master.dispatchMergingRegions(getRpcController(), request);
       }
     });
     return new DispatchMergingRegionsFuture(this, tableName, response);
@@ -1731,6 +1632,7 @@ public class HBaseAdmin implements Admin {
          Bytes.compareTo(hri.getStartKey(), splitPoint) == 0) {
        throw new IOException("should not give a splitkey which equals to startkey!");
     }
+    // TODO: There is no timeout on this controller. Set one!
     PayloadCarryingRpcController controller = rpcControllerFactory.newController();
     controller.setPriority(hri.getTable());
 
@@ -1746,21 +1648,16 @@ public class HBaseAdmin implements Admin {
       throw new IllegalArgumentException("the specified table name '" + tableName +
         "' doesn't match with the HTD one: " + htd.getTableName());
     }
-
     ModifyTableResponse response = executeCallable(
-      new MasterCallable<ModifyTableResponse>(getConnection()) {
+      new MasterCallable<ModifyTableResponse>(getConnection(), getRpcControllerFactory()) {
         @Override
-        public ModifyTableResponse call(int callTimeout) throws ServiceException {
-          PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-          controller.setCallTimeout(callTimeout);
-          controller.setPriority(tableName);
-
+        protected ModifyTableResponse rpcCall() throws Exception {
+          setPriority(tableName);
           ModifyTableRequest request = RequestConverter.buildModifyTableRequest(
             tableName, htd, ng.getNonceGroup(), ng.newNonce());
-          return master.modifyTable(controller, request);
+          return master.modifyTable(getRpcController(), request);
         }
       });
-
     return new ModifyTableFuture(this, tableName, response);
   }
 
@@ -1875,9 +1772,9 @@ public class HBaseAdmin implements Admin {
    */
   private TableName checkTableExists(final TableName tableName)
       throws IOException {
-    return executeCallable(new ConnectionCallable<TableName>(getConnection()) {
+    return executeCallable(new RpcRetryingCallable<TableName>() {
       @Override
-      public TableName call(int callTimeout) throws ServiceException, IOException {
+      protected TableName rpcCall(int callTimeout) throws Exception {
         if (!MetaTableAccessor.tableExists(connection, tableName)) {
           throw new TableNotFoundException(tableName);
         }
@@ -1888,13 +1785,11 @@ public class HBaseAdmin implements Admin {
 
   @Override
   public synchronized void shutdown() throws IOException {
-    executeCallable(new MasterCallable<Void>(getConnection()) {
+    executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
       @Override
-      public Void call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
-        controller.setPriority(HConstants.HIGH_QOS);
-        master.shutdown(controller, ShutdownRequest.newBuilder().build());
+      protected Void rpcCall() throws Exception {
+        setPriority(HConstants.HIGH_QOS);
+        master.shutdown(getRpcController(), ShutdownRequest.newBuilder().build());
         return null;
       }
     });
@@ -1902,13 +1797,11 @@ public class HBaseAdmin implements Admin {
 
   @Override
   public synchronized void stopMaster() throws IOException {
-    executeCallable(new MasterCallable<Void>(getConnection()) {
+    executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
       @Override
-      public Void call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
-        controller.setPriority(HConstants.HIGH_QOS);
-        master.stopMaster(controller, StopMasterRequest.newBuilder().build());
+      protected Void rpcCall() throws Exception {
+        setPriority(HConstants.HIGH_QOS);
+        master.stopMaster(getRpcController(), StopMasterRequest.newBuilder().build());
         return null;
       }
     });
@@ -1919,43 +1812,41 @@ public class HBaseAdmin implements Admin {
   throws IOException {
     String hostname = Addressing.parseHostname(hostnamePort);
     int port = Addressing.parsePort(hostnamePort);
-    AdminService.BlockingInterface admin =
+    final AdminService.BlockingInterface admin =
       this.connection.getAdmin(ServerName.valueOf(hostname, port, 0));
-    StopServerRequest request = RequestConverter.buildStopServerRequest(
-      "Called by admin client " + this.connection.toString());
+    // TODO: There is no timeout on this controller. Set one!
     PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-
     controller.setPriority(HConstants.HIGH_QOS);
+    StopServerRequest request = RequestConverter.buildStopServerRequest(
+        "Called by admin client " + this.connection.toString());
     try {
-      // TODO: this does not do retries, it should. Set priority and timeout in controller
       admin.stopServer(controller, request);
-    } catch (ServiceException se) {
-      throw ProtobufUtil.getRemoteException(se);
+    } catch (Exception e) {
+      throw ProtobufUtil.handleRemoteException(e);
     }
   }
 
   @Override
   public boolean isMasterInMaintenanceMode() throws IOException {
-    return executeCallable(new MasterCallable<IsInMaintenanceModeResponse>(getConnection()) {
+    return executeCallable(new MasterCallable<IsInMaintenanceModeResponse>(getConnection(),
+        this.rpcControllerFactory) {
       @Override
-      public IsInMaintenanceModeResponse call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
-        return master.isMasterInMaintenanceMode(
-          controller, IsInMaintenanceModeRequest.newBuilder().build());
+      protected IsInMaintenanceModeResponse rpcCall() throws Exception {
+        return master.isMasterInMaintenanceMode(getRpcController(),
+            IsInMaintenanceModeRequest.newBuilder().build());
       }
     }).getInMaintenanceMode();
   }
 
   @Override
   public ClusterStatus getClusterStatus() throws IOException {
-    return executeCallable(new MasterCallable<ClusterStatus>(getConnection()) {
+    return executeCallable(new MasterCallable<ClusterStatus>(getConnection(),
+        this.rpcControllerFactory) {
       @Override
-      public ClusterStatus call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
+      protected ClusterStatus rpcCall() throws Exception {
         GetClusterStatusRequest req = RequestConverter.buildGetClusterStatusRequest();
-        return ProtobufUtil.convert(master.getClusterStatus(controller, req).getClusterStatus());
+        return ProtobufUtil.convert(master.getClusterStatus(getRpcController(), req).
+            getClusterStatus());
       }
     });
   }
@@ -1996,19 +1887,15 @@ public class HBaseAdmin implements Admin {
   public Future<Void> createNamespaceAsync(final NamespaceDescriptor descriptor)
       throws IOException {
     CreateNamespaceResponse response =
-        executeCallable(new MasterCallable<CreateNamespaceResponse>(getConnection()) {
-          @Override
-          public CreateNamespaceResponse call(int callTimeout) throws Exception {
-            PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-            controller.setCallTimeout(callTimeout);
-            // TODO: set priority based on NS?
-            return master.createNamespace(controller,
-              CreateNamespaceRequest.newBuilder()
-              .setNamespaceDescriptor(ProtobufUtil
-                .toProtoNamespaceDescriptor(descriptor)).build()
-                );
-          }
-        });
+        executeCallable(new MasterCallable<CreateNamespaceResponse>(getConnection(),
+            getRpcControllerFactory()) {
+      @Override
+      protected CreateNamespaceResponse rpcCall() throws Exception {
+        return master.createNamespace(getRpcController(),
+          CreateNamespaceRequest.newBuilder().setNamespaceDescriptor(ProtobufUtil.
+              toProtoNamespaceDescriptor(descriptor)).build());
+      }
+    });
     return new NamespaceFuture(this, descriptor.getName(), response.getProcId()) {
       @Override
       public String getOperationType() {
@@ -2027,16 +1914,15 @@ public class HBaseAdmin implements Admin {
   public Future<Void> modifyNamespaceAsync(final NamespaceDescriptor descriptor)
       throws IOException {
     ModifyNamespaceResponse response =
-        executeCallable(new MasterCallable<ModifyNamespaceResponse>(getConnection()) {
-          @Override
-          public ModifyNamespaceResponse call(int callTimeout) throws Exception {
-            PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-            controller.setCallTimeout(callTimeout);
-            // TODO: set priority based on NS?
-            return master.modifyNamespace(controller, ModifyNamespaceRequest.newBuilder().
-              setNamespaceDescriptor(ProtobufUtil.toProtoNamespaceDescriptor(descriptor)).build());
-          }
-        });
+        executeCallable(new MasterCallable<ModifyNamespaceResponse>(getConnection(),
+            getRpcControllerFactory()) {
+      @Override
+      protected ModifyNamespaceResponse rpcCall() throws Exception {
+        // TODO: set priority based on NS?
+        return master.modifyNamespace(getRpcController(), ModifyNamespaceRequest.newBuilder().
+          setNamespaceDescriptor(ProtobufUtil.toProtoNamespaceDescriptor(descriptor)).build());
+       }
+    });
     return new NamespaceFuture(this, descriptor.getName(), response.getProcId()) {
       @Override
       public String getOperationType() {
@@ -2055,16 +1941,15 @@ public class HBaseAdmin implements Admin {
   public Future<Void> deleteNamespaceAsync(final String name)
       throws IOException {
     DeleteNamespaceResponse response =
-        executeCallable(new MasterCallable<DeleteNamespaceResponse>(getConnection()) {
-          @Override
-          public DeleteNamespaceResponse call(int callTimeout) throws Exception {
-            PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-            controller.setCallTimeout(callTimeout);
-            // TODO: set priority based on NS?
-            return master.deleteNamespace(controller, DeleteNamespaceRequest.newBuilder().
-              setNamespaceName(name).build());
-          }
-        });
+        executeCallable(new MasterCallable<DeleteNamespaceResponse>(getConnection(),
+            getRpcControllerFactory()) {
+      @Override
+      protected DeleteNamespaceResponse rpcCall() throws Exception {
+        // TODO: set priority based on NS?
+        return master.deleteNamespace(getRpcController(), DeleteNamespaceRequest.newBuilder().
+          setNamespaceName(name).build());
+        }
+      });
     return new NamespaceFuture(this, name, response.getProcId()) {
       @Override
       public String getOperationType() {
@@ -2075,100 +1960,90 @@ public class HBaseAdmin implements Admin {
 
   @Override
   public NamespaceDescriptor getNamespaceDescriptor(final String name) throws IOException {
-    return
-        executeCallable(new MasterCallable<NamespaceDescriptor>(getConnection()) {
-          @Override
-          public NamespaceDescriptor call(int callTimeout) throws Exception {
-            PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-            controller.setCallTimeout(callTimeout);
-            return ProtobufUtil.toNamespaceDescriptor(
-              master.getNamespaceDescriptor(controller, GetNamespaceDescriptorRequest.newBuilder().
-                setNamespaceName(name).build()).getNamespaceDescriptor());
-          }
-        });
+    return executeCallable(new MasterCallable<NamespaceDescriptor>(getConnection(),
+        getRpcControllerFactory()) {
+      @Override
+      protected NamespaceDescriptor rpcCall() throws Exception {
+        return ProtobufUtil.toNamespaceDescriptor(
+            master.getNamespaceDescriptor(getRpcController(),
+                GetNamespaceDescriptorRequest.newBuilder().
+                  setNamespaceName(name).build()).getNamespaceDescriptor());
+      }
+    });
   }
 
   @Override
   public NamespaceDescriptor[] listNamespaceDescriptors() throws IOException {
-    return
-        executeCallable(new MasterCallable<NamespaceDescriptor[]>(getConnection()) {
-          @Override
-          public NamespaceDescriptor[] call(int callTimeout) throws Exception {
-            PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-            controller.setCallTimeout(callTimeout);
-            List<HBaseProtos.NamespaceDescriptor> list =
-                master.listNamespaceDescriptors(controller,
-                  ListNamespaceDescriptorsRequest.newBuilder().build())
-                .getNamespaceDescriptorList();
-            NamespaceDescriptor[] res = new NamespaceDescriptor[list.size()];
-            for(int i = 0; i < list.size(); i++) {
-              res[i] = ProtobufUtil.toNamespaceDescriptor(list.get(i));
-            }
-            return res;
-          }
-        });
+    return executeCallable(new MasterCallable<NamespaceDescriptor[]>(getConnection(),
+        getRpcControllerFactory()) {
+      @Override
+      protected NamespaceDescriptor[] rpcCall() throws Exception {
+        List<HBaseProtos.NamespaceDescriptor> list =
+            master.listNamespaceDescriptors(getRpcController(),
+              ListNamespaceDescriptorsRequest.newBuilder().build()).getNamespaceDescriptorList();
+        NamespaceDescriptor[] res = new NamespaceDescriptor[list.size()];
+        for(int i = 0; i < list.size(); i++) {
+          res[i] = ProtobufUtil.toNamespaceDescriptor(list.get(i));
+        }
+        return res;
+      }
+    });
   }
 
   @Override
   public ProcedureInfo[] listProcedures() throws IOException {
-    return
-        executeCallable(new MasterCallable<ProcedureInfo[]>(getConnection()) {
-          @Override
-          public ProcedureInfo[] call(int callTimeout) throws Exception {
-            PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-            controller.setCallTimeout(callTimeout);
-            List<ProcedureProtos.Procedure> procList = master.listProcedures(
-              controller, ListProceduresRequest.newBuilder().build()).getProcedureList();
-            ProcedureInfo[] procInfoList = new ProcedureInfo[procList.size()];
-            for (int i = 0; i < procList.size(); i++) {
-              procInfoList[i] = ProcedureUtil.convert(procList.get(i));
-            }
-            return procInfoList;
-          }
-        });
+    return executeCallable(new MasterCallable<ProcedureInfo[]>(getConnection(),
+        getRpcControllerFactory()) {
+      @Override
+      protected ProcedureInfo[] rpcCall() throws Exception {
+        List<ProcedureProtos.Procedure> procList = master.listProcedures(
+            getRpcController(), ListProceduresRequest.newBuilder().build()).getProcedureList();
+        ProcedureInfo[] procInfoList = new ProcedureInfo[procList.size()];
+        for (int i = 0; i < procList.size(); i++) {
+          procInfoList[i] = ProcedureUtil.convert(procList.get(i));
+        }
+        return procInfoList;
+      }
+    });
   }
 
   @Override
   public HTableDescriptor[] listTableDescriptorsByNamespace(final String name) throws IOException {
-    return
-        executeCallable(new MasterCallable<HTableDescriptor[]>(getConnection()) {
-          @Override
-          public HTableDescriptor[] call(int callTimeout) throws Exception {
-            PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-            controller.setCallTimeout(callTimeout);
-            List<TableSchema> list =
-                master.listTableDescriptorsByNamespace(controller,
-                  ListTableDescriptorsByNamespaceRequest.newBuilder().setNamespaceName(name)
-                  .build()).getTableSchemaList();
-            HTableDescriptor[] res = new HTableDescriptor[list.size()];
-            for(int i=0; i < list.size(); i++) {
-
-              res[i] = ProtobufUtil.convertToHTableDesc(list.get(i));
-            }
-            return res;
-          }
-        });
+    return executeCallable(new MasterCallable<HTableDescriptor[]>(getConnection(),
+        getRpcControllerFactory()) {
+      @Override
+      protected HTableDescriptor[] rpcCall() throws Exception {
+        List<TableSchema> list =
+            master.listTableDescriptorsByNamespace(getRpcController(),
+                ListTableDescriptorsByNamespaceRequest.newBuilder().setNamespaceName(name)
+                .build()).getTableSchemaList();
+        HTableDescriptor[] res = new HTableDescriptor[list.size()];
+        for(int i=0; i < list.size(); i++) {
+
+          res[i] = ProtobufUtil.convertToHTableDesc(list.get(i));
+        }
+        return res;
+      }
+    });
   }
 
   @Override
   public TableName[] listTableNamesByNamespace(final String name) throws IOException {
-    return
-        executeCallable(new MasterCallable<TableName[]>(getConnection()) {
-          @Override
-          public TableName[] call(int callTimeout) throws Exception {
-            PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-            controller.setCallTimeout(callTimeout);
-            List<HBaseProtos.TableName> tableNames =
-              master.listTableNamesByNamespace(controller, ListTableNamesByNamespaceRequest.
+    return executeCallable(new MasterCallable<TableName[]>(getConnection(),
+        getRpcControllerFactory()) {
+      @Override
+      protected TableName[] rpcCall() throws Exception {
+        List<HBaseProtos.TableName> tableNames =
+            master.listTableNamesByNamespace(getRpcController(), ListTableNamesByNamespaceRequest.
                 newBuilder().setNamespaceName(name).build())
-                .getTableNameList();
-            TableName[] result = new TableName[tableNames.size()];
-            for (int i = 0; i < tableNames.size(); i++) {
-              result[i] = ProtobufUtil.toTableName(tableNames.get(i));
-            }
-            return result;
-          }
-        });
+            .getTableNameList();
+        TableName[] result = new TableName[tableNames.size()];
+        for (int i = 0; i < tableNames.size(); i++) {
+          result[i] = ProtobufUtil.toTableName(tableNames.get(i));
+        }
+        return result;
+      }
+    });
   }
 
   /**
@@ -2176,10 +2051,26 @@ public class HBaseAdmin implements Admin {
    * @param conf system configuration
    * @throws MasterNotRunningException if the master is not running
    * @throws ZooKeeperConnectionException if unable to connect to zookeeper
+   * @deprecated since hbase-2.0.0 because throws a ServiceException. We don't want to have
+   * protobuf as part of our public API. Use {@link #available(Configuration)}
    */
   // Used by tests and by the Merge tool. Merge tool uses it to figure if HBase is up or not.
+  // MOB uses it too.
+  // NOTE: hbase-2.0.0 removes ServiceException from the throw.
+  @Deprecated
   public static void checkHBaseAvailable(Configuration conf)
-  throws MasterNotRunningException, ZooKeeperConnectionException, ServiceException, IOException {
+  throws MasterNotRunningException, ZooKeeperConnectionException, IOException,
+  com.google.protobuf.ServiceException {
+    available(conf);
+  }
+
+  /**
+   * Is HBase available? Throw an exception if not.
+   * @param conf system configuration
+   * @throws ZooKeeperConnectionException if unable to connect to zookeeper]
+   */
+  public static void available(final Configuration conf)
+  throws ZooKeeperConnectionException, InterruptedIOException {
     Configuration copyOfConf = HBaseConfiguration.create(conf);
     // We set it to make it fail as soon as possible if HBase is not available
     copyOfConf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1);
@@ -2191,7 +2082,6 @@ public class HBaseAdmin implements Admin {
              (ClusterConnection) ConnectionFactory.createConnection(copyOfConf);
          ZooKeeperKeepAliveConnection zkw = ((ConnectionImplementation) connection).
              getKeepAliveZooKeeperWatcher();) {
-
       // This is NASTY. FIX!!!! Dependent on internal implementation! TODO
       zkw.getRecoverableZooKeeper().getZooKeeper().exists(zkw.baseZNode, false);
       connection.isMasterRunning();
@@ -2231,14 +2121,14 @@ public class HBaseAdmin implements Admin {
   @Override
   public HTableDescriptor[] getTableDescriptorsByTableName(final List<TableName> tableNames)
   throws IOException {
-    return executeCallable(new MasterCallable<HTableDescriptor[]>(getConnection()) {
+    return executeCallable(new MasterCallable<HTableDescriptor[]>(getConnection(),
+        getRpcControllerFactory()) {
       @Override
-      public HTableDescriptor[] call(int callTimeout) throws Exception {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
+      protected HTableDescriptor[] rpcCall() throws Exception {
         GetTableDescriptorsRequest req =
             RequestConverter.buildGetTableDescriptorsRequest(tableNames);
-          return ProtobufUtil.getHTableDescriptorArray(master.getTableDescriptors(controller, req));
+          return ProtobufUtil.
+              getHTableDescriptorArray(master.getTableDescriptors(getRpcController(), req));
       }
     });
   }
@@ -2276,15 +2166,14 @@ public class HBaseAdmin implements Admin {
 
   private RollWALWriterResponse rollWALWriterImpl(final ServerName sn) throws IOException,
       FailedLogCloseException {
-    AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
+    final AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
     RollWALWriterRequest request = RequestConverter.buildRollWALWriterRequest();
+    // TODO: There is no timeout on this controller. Set one!
     PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-
     try {
-      // TODO: this does not do retries, it should. Set priority and timeout in controller
       return admin.rollWALWriter(controller, request);
-    } catch (ServiceException se) {
-      throw ProtobufUtil.getRemoteException(se);
+    } catch (ServiceException e) {
+      throw ProtobufUtil.handleRemoteException(e);
     }
   }
 
@@ -2321,8 +2210,7 @@ public class HBaseAdmin implements Admin {
     }
     byte[][] regionsToFlush = new byte[regionCount][];
     for (int i = 0; i < regionCount; i++) {
-      ByteString region = response.getRegionToFlush(i);
-      regionsToFlush[i] = region.toByteArray();
+      regionsToFlush[i] = ProtobufUtil.toBytes(response.getRegionToFlush(i));
     }
     return regionsToFlush;
   }
@@ -2352,28 +2240,29 @@ public class HBaseAdmin implements Admin {
   @Override
   public CompactionState getCompactionStateForRegion(final byte[] regionName)
   throws IOException {
+    final Pair<HRegionInfo, ServerName> regionServerPair = getRegion(regionName);
+    if (regionServerPair == null) {
+      throw new IllegalArgumentException("Invalid region: " + Bytes.toStringBinary(regionName));
+    }
+    if (regionServerPair.getSecond() == null) {
+      throw new NoServerForRegionException(Bytes.toStringBinary(regionName));
+    }
+    ServerName sn = regionServerPair.getSecond();
+    final AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
+    // TODO: There is no timeout on this controller. Set one!
+    PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+    GetRegionInfoRequest request = RequestConverter.buildGetRegionInfoRequest(
+      regionServerPair.getFirst().getRegionName(), true);
+    GetRegionInfoResponse response;
     try {
-      Pair<HRegionInfo, ServerName> regionServerPair = getRegion(regionName);
-      if (regionServerPair == null) {
-        throw new IllegalArgumentException("Invalid region: " + Bytes.toStringBinary(regionName));
-      }
-      if (regionServerPair.getSecond() == null) {
-        throw new NoServerForRegionException(Bytes.toStringBinary(regionName));
-      }
-      ServerName sn = regionServerPair.getSecond();
-      AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
-      GetRegionInfoRequest request = RequestConverter.buildGetRegionInfoRequest(
-        regionServerPair.getFirst().getRegionName(), true);
-      PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-      // TODO: this does not do retries, it should. Set priority and timeout in controller
-      GetRegionInfoResponse response = admin.getRegionInfo(controller, request);
-      if (response.getCompactionState() != null) {
-        return ProtobufUtil.createCompactionState(response.getCompactionState());
-      }
-      return null;
-    } catch (ServiceException se) {
-      throw ProtobufUtil.getRemoteException(se);
+      response = admin.getRegionInfo(controller, request);
+    } catch (ServiceException e) {
+      throw ProtobufUtil.handleRemoteException(e);
+    }
+    if (response.getCompactionState() != null) {
+      return ProtobufUtil.createCompactionState(response.getCompactionState());
     }
+    return null;
   }
 
   @Override
@@ -2425,12 +2314,11 @@ public class HBaseAdmin implements Admin {
         throw (InterruptedIOException)new InterruptedIOException("Interrupted").initCause(e);
       }
       LOG.debug("Getting current status of snapshot from master...");
-      done = executeCallable(new MasterCallable<IsSnapshotDoneResponse>(getConnection()) {
+      done = executeCallable(new MasterCallable<IsSnapshotDoneResponse>(getConnection(),
+          getRpcControllerFactory()) {
         @Override
-        public IsSnapshotDoneResponse call(int callTimeout) throws ServiceException {
-          PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-          controller.setCallTimeout(callTimeout);
-          return master.isSnapshotDone(controller, request);
+        protected IsSnapshotDoneResponse rpcCall() throws Exception {
+          return master.isSnapshotDone(getRpcController(), request);
         }
       });
     }
@@ -2476,12 +2364,11 @@ public class HBaseAdmin implements Admin {
     final SnapshotRequest request = SnapshotRequest.newBuilder().setSnapshot(snapshot)
         .build();
     // run the snapshot on the master
-    return executeCallable(new MasterCallable<SnapshotResponse>(getConnection()) {
+    return executeCallable(new MasterCallable<SnapshotResponse>(getConnection(),
+        getRpcControllerFactory()) {
       @Override
-      public SnapshotResponse call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
-        return master.snapshot(controller, request);
+      protected SnapshotResponse rpcCall() throws Exception {
+        return master.snapshot(getRpcController(), request);
       }
     });
   }
@@ -2490,12 +2377,11 @@ public class HBaseAdmin implements Admin {
   public boolean isSnapshotFinished(final SnapshotDescription snapshotDesc)
       throws IOException, HBaseSnapshotException, UnknownSnapshotException {
     final HBaseProtos.SnapshotDescription snapshot = createHBaseProtosSnapshotDesc(snapshotDesc);
-    return executeCallable(new MasterCallable<IsSnapshotDoneResponse>(getConnection()) {
+    return executeCallable(new MasterCallable<IsSnapshotDoneResponse>(getConnection(),
+        getRpcControllerFactory()) {
       @Override
-      public IsSnapshotDoneResponse call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
-        return master.isSnapshotDone(controller,
+      protected IsSnapshotDoneResponse rpcCall() throws Exception {
+        return master.isSnapshotDone(getRpcController(),
           IsSnapshotDoneRequest.newBuilder().setSnapshot(snapshot).build());
       }
     }).getDone();
@@ -2674,12 +2560,10 @@ public class HBaseAdmin implements Admin {
         .setProcedure(builder.build()).build();
     // run the procedure on the master
     ExecProcedureResponse response = executeCallable(new MasterCallable<ExecProcedureResponse>(
-        getConnection()) {
+        getConnection(), getRpcControllerFactory()) {
       @Override
-      public ExecProcedureResponse call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
-        return master.execProcedureWithRet(controller, request);
+      protected ExecProcedureResponse rpcCall() throws Exception {
+        return master.execProcedureWithRet(getRpcController(), request);
       }
     });
 
@@ -2701,12 +2585,10 @@ public class HBaseAdmin implements Admin {
         .setProcedure(builder.build()).build();
     // run the procedure on the master
     ExecProcedureResponse response = executeCallable(new MasterCallable<ExecProcedureResponse>(
-        getConnection()) {
+        getConnection(), getRpcControllerFactory()) {
       @Override
-      public ExecProcedureResponse call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
-        return master.execProcedure(controller, request);
+      protected ExecProcedureResponse rpcCall() throws Exception {
+        return master.execProcedure(getRpcController(), request);
       }
     });
 
@@ -2750,12 +2632,10 @@ public class HBaseAdmin implements Admin {
     }
     final ProcedureDescription desc = builder.build();
     return executeCallable(
-        new MasterCallable<IsProcedureDoneResponse>(getConnection()) {
+        new MasterCallable<IsProcedureDoneResponse>(getConnection(), getRpcControllerFactory()) {
           @Override
-          public IsProcedureDoneResponse call(int callTimeout) throws ServiceException {
-            PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-            controller.setCallTimeout(callTimeout);
-            return master.isProcedureDone(controller, IsProcedureDoneRequest
+          protected IsProcedureDoneResponse rpcCall() throws Exception {
+            return master.isProcedureDone(getRpcController(), IsProcedureDoneRequest
                 .newBuilder().setProcedure(desc).build());
           }
         }).getDone();
@@ -2781,17 +2661,15 @@ public class HBaseAdmin implements Admin {
     ClientSnapshotDescriptionUtils.assertSnapshotRequestIsValid(snapshot);
 
     RestoreSnapshotResponse response = executeCallable(
-        new MasterCallable<RestoreSnapshotResponse>(getConnection()) {
+        new MasterCallable<RestoreSnapshotResponse>(getConnection(), getRpcControllerFactory()) {
       @Override
-      public RestoreSnapshotResponse call(int callTimeout) throws ServiceException {
+      protected RestoreSnapshotResponse rpcCall() throws Exception {
         final RestoreSnapshotRequest request = RestoreSnapshotRequest.newBuilder()
             .setSnapshot(snapshot)
             .setNonceGroup(ng.getNonceGroup())
             .setNonce(ng.newNonce())
             .build();
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
-        return master.restoreSnapshot(controller, request);
+        return master.restoreSnapshot(getRpcController(), request);
       }
     });
 
@@ -2828,13 +2706,13 @@ public class HBaseAdmin implements Admin {
 
   @Override
   public List<SnapshotDescription> listSnapshots() throws IOException {
-    return executeCallable(new MasterCallable<List<SnapshotDescription>>(getConnection()) {
+    return executeCallable(new MasterCallable<List<SnapshotDescription>>(getConnection(),
+        getRpcControllerFactory()) {
       @Override
-      public List<SnapshotDescription> call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
+      protected List<SnapshotDescription> rpcCall() throws Exception {
         List<HBaseProtos.SnapshotDescription> snapshotsList = master
-            .getCompletedSnapshots(controller, GetCompletedSnapshotsRequest.newBuilder().build())
+            .getCompletedSnapshots(getRpcController(),
+                GetCompletedSnapshotsRequest.newBuilder().build())
             .getSnapshotsList();
         List<SnapshotDescription> result = new ArrayList<SnapshotDescription>(snapshotsList.size());
         for (HBaseProtos.SnapshotDescription snapshot : snapshotsList) {
@@ -2897,14 +2775,11 @@ public class HBaseAdmin implements Admin {
     // make sure the snapshot is possibly valid
     TableName.isLegalFullyQualifiedTableName(Bytes.toBytes(snapshotName));
     // do the delete
-    executeCallable(new MasterCallable<Void>(getConnection()) {
+    executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
       @Override
-      public Void call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
-        master.deleteSnapshot(controller,
-          DeleteSnapshotRequest.newBuilder().
-              setSnapshot(
+      protected Void rpcCall() throws Exception {
+        master.deleteSnapshot(getRpcController(),
+          DeleteSnapshotRequest.newBuilder().setSnapshot(
                 HBaseProtos.SnapshotDescription.newBuilder().setName(snapshotName).build())
               .build()
         );
@@ -2933,12 +2808,10 @@ public class HBaseAdmin implements Admin {
   }
 
   private void internalDeleteSnapshot(final SnapshotDescription snapshot) throws IOException {
-    executeCallable(new MasterCallable<Void>(getConnection()) {
+    executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
       @Override
-      public Void call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
-        this.master.deleteSnapshot(controller, DeleteSnapshotRequest.newBuilder()
+      protected Void rpcCall() throws Exception {
+        this.master.deleteSnapshot(getRpcController(), DeleteSnapshotRequest.newBuilder()
           .setSnapshot(createHBaseProtosSnapshotDesc(snapshot)).build());
         return null;
       }
@@ -2967,12 +2840,10 @@ public class HBaseAdmin implements Admin {
 
   @Override
   public void setQuota(final QuotaSettings quota) throws IOException {
-    executeCallable(new MasterCallable<Void>(getConnection()) {
+    executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
       @Override
-      public Void call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
-        this.master.setQuota(controller, QuotaSettings.buildSetQuotaRequestProto(quota));
+      protected Void rpcCall() throws Exception {
+        this.master.setQuota(getRpcController(), QuotaSettings.buildSetQuotaRequestProto(quota));
         return null;
       }
     });
@@ -2989,8 +2860,8 @@ public class HBaseAdmin implements Admin {
   }
 
   static private <C extends RetryingCallable<V> & Closeable, V> V executeCallable(C callable,
-             RpcRetryingCallerFactory rpcCallerFactory, int operationTimeout,
-      int rpcTimeout) throws IOException {
+             RpcRetryingCallerFactory rpcCallerFactory, int operationTimeout, int rpcTimeout)
+  throws IOException {
     RpcRetryingCaller<V> caller = rpcCallerFactory.newCaller(rpcTimeout);
     try {
       return caller.callWithRetries(callable, operationTimeout);
@@ -3008,7 +2879,6 @@ public class HBaseAdmin implements Admin {
    * Simple {@link Abortable}, throwing RuntimeException on abort.
    */
   private static class ThrowableAbortable implements Abortable {
-
     @Override
     public void abort(String why, Throwable e) {
       throw new RuntimeException(why, e);
@@ -3026,13 +2896,16 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public void updateConfiguration(ServerName server) throws IOException {
-    try {
-      this.connection.getAdmin(server).updateConfiguration(null,
-        UpdateConfigurationRequest.getDefaultInstance());
-    } catch (ServiceException e) {
-      throw ProtobufUtil.getRemoteException(e);
-    }
+  public void updateConfiguration(final ServerName server) throws IOException {
+    final AdminService.BlockingInterface admin = this.connection.getAdmin(server);
+    Callable<Void> callable = new Callable<Void>() {
+      @Override
+      public Void call() throws Exception {
+        admin.updateConfiguration(null, UpdateConfigurationRequest.getDefaultInstance());
+        return null;
+      }
+    };
+    ProtobufUtil.call(callable);
   }
 
   @Override
@@ -3045,8 +2918,7 @@ public class HBaseAdmin implements Admin {
   @Override
   public int getMasterInfoPort() throws IOException {
     // TODO: Fix!  Reaching into internal implementation!!!!
-    ConnectionImplementation connection =
-        (ConnectionImplementation)this.connection;
+    ConnectionImplementation connection = (ConnectionImplementation)this.connection;
     ZooKeeperKeepAliveConnection zkw = connection.getKeepAliveZooKeeperWatcher();
     try {
       return MasterAddressTracker.getMasterInfoPort(zkw);
@@ -3057,8 +2929,7 @@ public class HBaseAdmin implements Admin {
 
   private ServerName getMasterAddress() throws IOException {
     // TODO: Fix!  Reaching into internal implementation!!!!
-    ConnectionImplementation connection =
-            (ConnectionImplementation)this.connection;
+    ConnectionImplementation connection = (ConnectionImplementation)this.connection;
     ZooKeeperKeepAliveConnection zkw = connection.getKeepAliveZooKeeperWatcher();
     try {
       return MasterAddressTracker.getMasterAddress(zkw);
@@ -3069,33 +2940,27 @@ public class HBaseAdmin implements Admin {
 
   @Override
   public long getLastMajorCompactionTimestamp(final TableName tableName) throws IOException {
-    return executeCallable(new MasterCallable<Long>(getConnection()) {
+    return executeCallable(new MasterCallable<Long>(getConnection(), getRpcControllerFactory()) {
       @Override
-      public Long call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
+      protected Long rpcCall() throws Exception {
         MajorCompactionTimestampRequest req =
             MajorCompactionTimestampRequest.newBuilder()
                 .setTableName(ProtobufUtil.toProtoTableName(tableName)).build();
-        return master.getLastMajorCompactionTimestamp(controller, req).getCompactionTimestamp();
+        return master.getLastMajorCompactionTimestamp(getRpcController(), req).
+            getCompactionTimestamp();
       }
     });
   }
 
   @Override
   public long getLastMajorCompactionTimestampForRegion(final byte[] regionName) throws IOException {
-    return executeCallable(new MasterCallable<Long>(getConnection()) {
+    return executeCallable(new MasterCallable<Long>(getConnection(), getRpcControllerFactory()) {
       @Override
-      public Long call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
+      protected Long rpcCall() throws Exception {
         MajorCompactionTimestampForRegionRequest req =
-            MajorCompactionTimestampForRegionRequest
-                .newBuilder()
-                .setRegion(
-                  RequestConverter
+            MajorCompactionTimestampForRegionRequest.newBuilder().setRegion(RequestConverter
                       .buildRegionSpecifier(RegionSpecifierType.REGION_NAME, regionName)).build();
-        return master.getLastMajorCompactionTimestampForRegion(controller, req)
+        return master.getLastMajorCompactionTimestampForRegion(getRpcController(), req)
             .getCompactionTimestamp();
       }
     });
@@ -3134,32 +2999,36 @@ public class HBaseAdmin implements Admin {
   @Override
   public void majorCompact(final TableName tableName, CompactType compactType)
           throws IOException, InterruptedException {
-      compact(tableName, null, true, compactType);
+    compact(tableName, null, true, compactType);
   }
 
   /**
    * {@inheritDoc}
    */
   @Override
-  public CompactionState getCompactionState(TableName tableName,
+  public CompactionState getCompactionState(final TableName tableName,
     CompactType compactType) throws IOException {
     AdminProtos.GetRegionInfoResponse.CompactionState state =
         AdminProtos.GetRegionInfoResponse.CompactionState.NONE;
     checkTableExists(tableName);
-    PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+    // TODO: There is no timeout on this controller. Set one!
+    final PayloadCarryingRpcController rpcController = rpcControllerFactory.newController();
     switch (compactType) {
       case MOB:
-        try {
-          ServerName master = getMasterAddress();
-          HRegionInfo info = getMobRegionInfo(tableName);
-          GetRegionInfoRequest request = RequestConverter.buildGetRegionInfoRequest(
-                  info.getRegionName(), true);
-          GetRegionInfoResponse response = this.connection.getAdmin(master)
-                  .getRegionInfo(controller, request);
-          state = response.getCompactionState();
-        } catch (ServiceException se) {
-          throw ProtobufUtil.getRemoteException(se);
-        }
+        final AdminProtos.AdminService.BlockingInterface masterAdmin =
+          this.connection.getAdmin(getMasterAddress());
+        Callable<AdminProtos.GetRegionInfoResponse.CompactionState> callable =
+            new Callable<AdminProtos.GetRegionInfoResponse.CompactionState>() {
+          @Override
+          public AdminProtos.GetRegionInfoResponse.CompactionState call() throws Exception {
+            HRegionInfo info = getMobRegionInfo(tableName);
+            GetRegionInfoRequest request = RequestConverter.buildGetRegionInfoRequest(
+                info.getRegionName(), true);
+            GetRegionInfoResponse response = masterAdmin.getRegionInfo(rpcController, request);
+            return response.getCompactionState();
+          }
+        };
+        state = ProtobufUtil.call(callable);
         break;
       case NORMAL:
       default:
@@ -3173,15 +3042,23 @@ public class HBaseAdmin implements Admin {
           } else {
             pairs = MetaTableAccessor.getTableRegionsAndLocations(connection, tableName);
           }
-          for (Pair<HRegionInfo, ServerName> pair : pairs) {
+          for (Pair<HRegionInfo, ServerName> pair: pairs) {
             if (pair.getFirst().isOffline()) continue;
             if (pair.getSecond() == null) continue;
+            final ServerName sn = pair.getSecond();
+            final byte [] regionName = pair.getFirst().getRegionName();
+            final AdminService.BlockingInterface snAdmin = this.connection.getAdmin(sn);
             try {
-              ServerName sn = pair.getSecond();
-              AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
-              GetRegionInfoRequest request = RequestConverter.buildGetRegionInfoRequest(
-                      pair.getFirst().getRegionName(), true);
-              GetRegionInfoResponse response = admin.getRegionInfo(controller, request);
+              Callable<GetRegionInfoResponse> regionInfoCallable =
+                  new Callable<GetRegionInfoResponse>() {
+                @Override
+                public GetRegionInfoResponse call() throws Exception {
+                  GetRegionInfoRequest request = RequestConverter.buildGetRegionInfoRequest(
+                      regionName, true);
+                  return snAdmin.getRegionInfo(rpcController, request);
+                }
+              };
+              GetRegionInfoResponse response = ProtobufUtil.call(regionInfoCallable);
               switch (response.getCompactionState()) {
                 case MAJOR_AND_MINOR:
                   return CompactionState.MAJOR_AND_MINOR;
@@ -3217,8 +3094,6 @@ public class HBaseAdmin implements Admin {
               }
             }
           }
-        } catch (ServiceException se) {
-          throw ProtobufUtil.getRemoteException(se);
         } finally {
           if (zookeeper != null) {
             zookeeper.close();
@@ -3283,12 +3158,10 @@ public class HBaseAdmin implements Admin {
     protected AbortProcedureResponse abortProcedureResult(
         final AbortProcedureRequest request) throws IOException {
       return admin.executeCallable(new MasterCallable<AbortProcedureResponse>(
-          admin.getConnection()) {
+          admin.getConnection(), admin.getRpcControllerFactory()) {
         @Override
-        public AbortProcedureResponse call(int callTimeout) throws ServiceException {
-          PayloadCarryingRpcController controller = admin.getRpcControllerFactory().newController();
-          controller.setCallTimeout(callTimeout);
-          return master.abortProcedure(controller, request);
+        protected AbortProcedureResponse rpcCall() throws Exception {
+          return master.abortProcedure(getRpcController(), request);
         }
       });
     }
@@ -3401,10 +3274,10 @@ public class HBaseAdmin implements Admin {
     protected GetProcedureResultResponse getProcedureResult(final GetProcedureResultRequest request)
         throws IOException {
       return admin.executeCallable(new MasterCallable<GetProcedureResultResponse>(
-          admin.getConnection()) {
+          admin.getConnection(), admin.getRpcControllerFactory()) {
         @Override
-        public GetProcedureResultResponse call(int callTimeout) throws ServiceException {
-          return master.getProcedureResult(null, request);
+        protected GetProcedureResultResponse rpcCall() throws Exception {
+          return master.getProcedureResult(getRpcController(), request);
         }
       });
     }
@@ -3699,14 +3572,13 @@ public class HBaseAdmin implements Admin {
   @Override
   public List<SecurityCapability> getSecurityCapabilities() throws IOException {
     try {
-      return executeCallable(new MasterCallable<List<SecurityCapability>>(getConnection()) {
+   

<TRUNCATED>

[04/50] [abbrv] hbase git commit: HBASE-9899 for idempotent operation dups, return the result instead of throwing conflict exception (Guanghao Zhang)

Posted by sy...@apache.org.
HBASE-9899 for idempotent operation dups, return the result instead of throwing conflict exception (Guanghao Zhang)


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

Branch: refs/heads/hbase-12439
Commit: 975f0dd958debcdd842a95f8e9f7458689414fbf
Parents: 550b937
Author: stack <st...@apache.org>
Authored: Thu Aug 4 12:40:19 2016 -0700
Committer: stack <st...@apache.org>
Committed: Thu Aug 4 12:40:19 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/protobuf/ProtobufUtil.java     | 57 ++++++++++++++++++++
 .../hadoop/hbase/regionserver/HRegion.java      | 53 ++++++++++++++----
 .../hbase/regionserver/RSRpcServices.java       | 55 +++++++++++--------
 .../hadoop/hbase/regionserver/Region.java       | 11 ++++
 .../hbase/regionserver/ServerNonceManager.java  | 43 +++++++++++++++
 .../hbase/client/HConnectionTestingUtility.java | 44 +++++++++++++++
 .../hadoop/hbase/client/TestFromClientSide.java | 45 ++++++++++++++++
 .../client/TestIncrementsFromClientSide.java    | 48 +++++++++++++++++
 .../hadoop/hbase/client/TestMultiParallel.java  | 14 +++--
 .../TestScannerHeartbeatMessages.java           |  2 +-
 10 files changed, 332 insertions(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/975f0dd9/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
index b3bf041..5ba0572 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
@@ -859,6 +859,63 @@ public final class ProtobufUtil {
   }
 
   /**
+   * Convert a protocol buffer Mutate to a Get.
+   * @param proto the protocol buffer Mutate to convert.
+   * @param cellScanner
+   * @return the converted client get.
+   * @throws IOException
+   */
+  public static Get toGet(final MutationProto proto, final CellScanner cellScanner)
+      throws IOException {
+    MutationType type = proto.getMutateType();
+    assert type == MutationType.INCREMENT || type == MutationType.APPEND : type.name();
+    byte[] row = proto.hasRow() ? proto.getRow().toByteArray() : null;
+    Get get = null;
+    int cellCount = proto.hasAssociatedCellCount() ? proto.getAssociatedCellCount() : 0;
+    if (cellCount > 0) {
+      // The proto has metadata only and the data is separate to be found in the cellScanner.
+      if (cellScanner == null) {
+        throw new DoNotRetryIOException("Cell count of " + cellCount + " but no cellScanner: "
+            + TextFormat.shortDebugString(proto));
+      }
+      for (int i = 0; i < cellCount; i++) {
+        if (!cellScanner.advance()) {
+          throw new DoNotRetryIOException("Cell count of " + cellCount + " but at index " + i
+              + " no cell returned: " + TextFormat.shortDebugString(proto));
+        }
+        Cell cell = cellScanner.current();
+        if (get == null) {
+          get = new Get(Bytes.copy(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength()));
+        }
+        get.addColumn(
+          Bytes.copy(cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength()),
+          Bytes.copy(cell.getQualifierArray(), cell.getQualifierOffset(),
+            cell.getQualifierLength()));
+      }
+    } else {
+      get = new Get(row);
+      for (ColumnValue column : proto.getColumnValueList()) {
+        byte[] family = column.getFamily().toByteArray();
+        for (QualifierValue qv : column.getQualifierValueList()) {
+          byte[] qualifier = qv.getQualifier().toByteArray();
+          if (!qv.hasValue()) {
+            throw new DoNotRetryIOException("Missing required field: qualifier value");
+          }
+          get.addColumn(family, qualifier);
+        }
+      }
+    }
+    if (proto.hasTimeRange()) {
+      TimeRange timeRange = protoToTimeRange(proto.getTimeRange());
+      get.setTimeRange(timeRange.getMin(), timeRange.getMax());
+    }
+    for (NameBytesPair attribute : proto.getAttributeList()) {
+      get.setAttribute(attribute.getName(), attribute.getValue().toByteArray());
+    }
+    return get;
+  }
+
+  /**
    * Convert a client Scan to a protocol buffer Scan
    *
    * @param scan the client Scan to convert

http://git-wip-us.apache.org/repos/asf/hbase/blob/975f0dd9/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index b7950df..86c02ea 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -2617,6 +2617,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   @Override
   public RegionScanner getScanner(Scan scan, List<KeyValueScanner> additionalScanners)
       throws IOException {
+    return getScanner(scan, additionalScanners, HConstants.NO_NONCE, HConstants.NO_NONCE);
+  }
+
+  private RegionScanner getScanner(Scan scan, List<KeyValueScanner> additionalScanners,
+      long nonceGroup, long nonce) throws IOException {
     startRegionOperation(Operation.SCAN);
     try {
       // Verify families are all valid
@@ -2630,7 +2635,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
           checkFamily(family);
         }
       }
-      return instantiateRegionScanner(scan, additionalScanners);
+      return instantiateRegionScanner(scan, additionalScanners, nonceGroup, nonce);
     } finally {
       closeRegionOperation(Operation.SCAN);
     }
@@ -2638,13 +2643,19 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
 
   protected RegionScanner instantiateRegionScanner(Scan scan,
       List<KeyValueScanner> additionalScanners) throws IOException {
+    return instantiateRegionScanner(scan, additionalScanners, HConstants.NO_NONCE,
+      HConstants.NO_NONCE);
+  }
+
+  protected RegionScanner instantiateRegionScanner(Scan scan,
+      List<KeyValueScanner> additionalScanners, long nonceGroup, long nonce) throws IOException {
     if (scan.isReversed()) {
       if (scan.getFilter() != null) {
         scan.getFilter().setReversed(true);
       }
       return new ReversedRegionScannerImpl(scan, additionalScanners, this);
     }
-    return new RegionScannerImpl(scan, additionalScanners, this);
+    return new RegionScannerImpl(scan, additionalScanners, this, nonceGroup, nonce);
   }
 
   @Override
@@ -5592,6 +5603,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
 
     RegionScannerImpl(Scan scan, List<KeyValueScanner> additionalScanners, HRegion region)
         throws IOException {
+      this(scan, additionalScanners, region, HConstants.NO_NONCE, HConstants.NO_NONCE);
+    }
+
+    RegionScannerImpl(Scan scan, List<KeyValueScanner> additionalScanners, HRegion region,
+        long nonceGroup, long nonce) throws IOException {
       this.region = region;
       this.maxResultSize = scan.getMaxResultSize();
       if (scan.hasFilter()) {
@@ -5621,15 +5637,25 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       // getSmallestReadPoint, before scannerReadPoints is updated.
       IsolationLevel isolationLevel = scan.getIsolationLevel();
       synchronized(scannerReadPoints) {
-        this.readPt = getReadPoint(isolationLevel);
+        if (nonce == HConstants.NO_NONCE || rsServices == null
+            || rsServices.getNonceManager() == null) {
+          this.readPt = getReadPoint(isolationLevel);
+        } else {
+          this.readPt = rsServices.getNonceManager().getMvccFromOperationContext(nonceGroup, nonce);
+        }
         scannerReadPoints.put(this, this.readPt);
       }
 
+      initializeScanners(scan, additionalScanners);
+    }
+
+    protected void initializeScanners(Scan scan, List<KeyValueScanner> additionalScanners)
+        throws IOException {
       // Here we separate all scanners into two lists - scanner that provide data required
       // by the filter to operate (scanners list) and all others (joinedScanners list).
       List<KeyValueScanner> scanners = new ArrayList<KeyValueScanner>(scan.getFamilyMap().size());
-      List<KeyValueScanner> joinedScanners
-        = new ArrayList<KeyValueScanner>(scan.getFamilyMap().size());
+      List<KeyValueScanner> joinedScanners =
+          new ArrayList<KeyValueScanner>(scan.getFamilyMap().size());
       // Store all already instantiated scanners for exception handling
       List<KeyValueScanner> instantiatedScanners = new ArrayList<KeyValueScanner>();
       // handle additionalScanners
@@ -6795,15 +6821,15 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     return Result.create(results, get.isCheckExistenceOnly() ? !results.isEmpty() : null, stale);
   }
 
-   void prepareGet(final Get get) throws IOException, NoSuchColumnFamilyException {
+  void prepareGet(final Get get) throws IOException, NoSuchColumnFamilyException {
     checkRow(get.getRow(), "Get");
     // Verify families are all valid
     if (get.hasFamilies()) {
-      for (byte [] family: get.familySet()) {
+      for (byte[] family : get.familySet()) {
         checkFamily(family);
       }
     } else { // Adding all families to scanner
-      for (byte[] family: this.htableDescriptor.getFamiliesKeys()) {
+      for (byte[] family : this.htableDescriptor.getFamiliesKeys()) {
         get.addFamily(family);
       }
     }
@@ -6811,7 +6837,12 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
 
   @Override
   public List<Cell> get(Get get, boolean withCoprocessor) throws IOException {
+    return get(get, withCoprocessor, HConstants.NO_NONCE, HConstants.NO_NONCE);
+  }
 
+  @Override
+  public List<Cell> get(Get get, boolean withCoprocessor, long nonceGroup, long nonce)
+      throws IOException {
     List<Cell> results = new ArrayList<Cell>();
 
     // pre-get CP hook
@@ -6825,7 +6856,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
 
     RegionScanner scanner = null;
     try {
-      scanner = getScanner(scan);
+      scanner = getScanner(scan, null, nonceGroup, nonce);
       scanner.next(results);
     } finally {
       if (scanner != null)
@@ -7168,6 +7199,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
               applyToMemstore(e.getKey(), e.getValue(), true, false, sequenceId);
         }
         mvcc.completeAndWait(writeEntry);
+        if (rsServices != null && rsServices.getNonceManager() != null) {
+          rsServices.getNonceManager().addMvccToOperationContext(nonceGroup, nonce,
+            writeEntry.getWriteNumber());
+        }
         writeEntry = null;
       } finally {
         this.updatesLock.readLock().unlock();

http://git-wip-us.apache.org/repos/asf/hbase/blob/975f0dd9/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index 9cfc5df..f9b78e1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -73,7 +73,6 @@ import org.apache.hadoop.hbase.client.VersionInfoUtil;
 import org.apache.hadoop.hbase.conf.ConfigurationObserver;
 import org.apache.hadoop.hbase.exceptions.FailedSanityCheckException;
 import org.apache.hadoop.hbase.exceptions.MergeRegionException;
-import org.apache.hadoop.hbase.exceptions.OperationConflictException;
 import org.apache.hadoop.hbase.exceptions.OutOfOrderScannerNextException;
 import org.apache.hadoop.hbase.filter.ByteArrayComparable;
 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
@@ -426,11 +425,11 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
    * Starts the nonce operation for a mutation, if needed.
    * @param mutation Mutation.
    * @param nonceGroup Nonce group from the request.
-   * @returns Nonce used (can be NO_NONCE).
+   * @returns whether to proceed this mutation.
    */
-  private long startNonceOperation(final MutationProto mutation, long nonceGroup)
-      throws IOException, OperationConflictException {
-    if (regionServer.nonceManager == null || !mutation.hasNonce()) return HConstants.NO_NONCE;
+  private boolean startNonceOperation(final MutationProto mutation, long nonceGroup)
+      throws IOException {
+    if (regionServer.nonceManager == null || !mutation.hasNonce()) return true;
     boolean canProceed = false;
     try {
       canProceed = regionServer.nonceManager.startOperation(
@@ -438,14 +437,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     } catch (InterruptedException ex) {
       throw new InterruptedIOException("Nonce start operation interrupted");
     }
-    if (!canProceed) {
-      // TODO: instead, we could convert append/increment to get w/mvcc
-      String message = "The operation with nonce {" + nonceGroup + ", " + mutation.getNonce()
-        + "} on row [" + Bytes.toString(mutation.getRow().toByteArray())
-        + "] may have already completed";
-      throw new OperationConflictException(message);
-    }
-    return mutation.getNonce();
+    return canProceed;
   }
 
   /**
@@ -614,23 +606,33 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
    * bypassed as indicated by RegionObserver, null otherwise
    * @throws IOException
    */
-  private Result append(final Region region, final OperationQuota quota, final MutationProto m,
-      final CellScanner cellScanner, long nonceGroup) throws IOException {
+  private Result append(final Region region, final OperationQuota quota,
+      final MutationProto mutation, final CellScanner cellScanner, long nonceGroup)
+      throws IOException {
     long before = EnvironmentEdgeManager.currentTime();
-    Append append = ProtobufUtil.toAppend(m, cellScanner);
+    Append append = ProtobufUtil.toAppend(mutation, cellScanner);
     quota.addMutation(append);
     Result r = null;
     if (region.getCoprocessorHost() != null) {
       r = region.getCoprocessorHost().preAppend(append);
     }
     if (r == null) {
-      long nonce = startNonceOperation(m, nonceGroup);
+      boolean canProceed = startNonceOperation(mutation, nonceGroup);
       boolean success = false;
       try {
-        r = region.append(append, nonceGroup, nonce);
+        if (canProceed) {
+          r = region.append(append, nonceGroup, mutation.getNonce());
+        } else {
+          // convert duplicate append to get
+          List<Cell> results = region.get(ProtobufUtil.toGet(mutation, cellScanner), false,
+            nonceGroup, mutation.getNonce());
+          r = Result.create(results);
+        }
         success = true;
       } finally {
-        endNonceOperation(m, nonceGroup, success);
+        if (canProceed) {
+          endNonceOperation(mutation, nonceGroup, success);
+        }
       }
       if (region.getCoprocessorHost() != null) {
         region.getCoprocessorHost().postAppend(append, r);
@@ -662,13 +664,22 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
       r = region.getCoprocessorHost().preIncrement(increment);
     }
     if (r == null) {
-      long nonce = startNonceOperation(mutation, nonceGroup);
+      boolean canProceed = startNonceOperation(mutation, nonceGroup);
       boolean success = false;
       try {
-        r = region.increment(increment, nonceGroup, nonce);
+        if (canProceed) {
+          r = region.increment(increment, nonceGroup, mutation.getNonce());
+        } else {
+          // convert duplicate increment to get
+          List<Cell> results = region.get(ProtobufUtil.toGet(mutation, cells), false, nonceGroup,
+            mutation.getNonce());
+          r = Result.create(results);
+        }
         success = true;
       } finally {
-        endNonceOperation(mutation, nonceGroup, success);
+        if (canProceed) {
+          endNonceOperation(mutation, nonceGroup, success);
+        }
       }
       if (region.getCoprocessorHost() != null) {
         r = region.getCoprocessorHost().postIncrement(increment, r);

http://git-wip-us.apache.org/repos/asf/hbase/blob/975f0dd9/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java
index 9b1f82a..efd68b8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java
@@ -395,6 +395,17 @@ public interface Region extends ConfigurationObserver {
   List<Cell> get(Get get, boolean withCoprocessor) throws IOException;
 
   /**
+   * Do a get for duplicate non-idempotent operation.
+   * @param get query parameters.
+   * @param withCoprocessor
+   * @param nonceGroup Nonce group.
+   * @param nonce Nonce.
+   * @return list of cells resulting from the operation
+   * @throws IOException
+   */
+  List<Cell> get(Get get, boolean withCoprocessor, long nonceGroup, long nonce) throws IOException;
+
+  /**
    * Return an iterator that scans over the HRegion, returning the indicated
    * columns and rows specified by the {@link Scan}.
    * <p>

http://git-wip-us.apache.org/repos/asf/hbase/blob/975f0dd9/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ServerNonceManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ServerNonceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ServerNonceManager.java
index b2b656b..459b69a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ServerNonceManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ServerNonceManager.java
@@ -62,6 +62,8 @@ public class ServerNonceManager {
     private static final long WAITING_BIT = 4;
     private static final long ALL_FLAG_BITS = WAITING_BIT | STATE_BITS;
 
+    private static long mvcc;
+
     @Override
     public String toString() {
       return "[state " + getState() + ", hasWait " + hasWait() + ", activity "
@@ -98,6 +100,14 @@ public class ServerNonceManager {
       return getActivityTime() < (minRelevantTime & (~0l >>> 3));
     }
 
+    public void setMvcc(long mvcc) {
+      this.mvcc = mvcc;
+    }
+
+    public long getMvcc() {
+      return this.mvcc;
+    }
+
     private long getActivityTime() {
       return this.data >>> 3;
     }
@@ -192,6 +202,39 @@ public class ServerNonceManager {
   }
 
   /**
+   * Store the write point in OperationContext when the operation succeed.
+   * @param group Nonce group.
+   * @param nonce Nonce.
+   * @param mvcc Write point of the succeed operation.
+   */
+  public void addMvccToOperationContext(long group, long nonce, long mvcc) {
+    if (nonce == HConstants.NO_NONCE) {
+      return;
+    }
+    NonceKey nk = new NonceKey(group, nonce);
+    OperationContext result = nonces.get(nk);
+    assert result != null;
+    synchronized (result) {
+      result.setMvcc(mvcc);
+    }
+  }
+
+  /**
+   * Return the write point of the previous succeed operation.
+   * @param group Nonce group.
+   * @param nonce Nonce.
+   * @return write point of the previous succeed operation.
+   */
+  public long getMvccFromOperationContext(long group, long nonce) {
+    if (nonce == HConstants.NO_NONCE) {
+      return Long.MAX_VALUE;
+    }
+    NonceKey nk = new NonceKey(group, nonce);
+    OperationContext result = nonces.get(nk);
+    return result == null ? Long.MAX_VALUE : result.getMvcc();
+  }
+
+  /**
    * Reports the operation from WAL during replay.
    * @param group Nonce group.
    * @param nonce Nonce.

http://git-wip-us.apache.org/repos/asf/hbase/blob/975f0dd9/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java
index c8ccd2a..265e3c1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hbase.client;
 
 import java.io.IOException;
+import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.RegionLocations;
@@ -28,6 +29,10 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
+import org.apache.hadoop.hbase.util.Threads;
+import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.mockito.Mockito;
 
@@ -150,4 +155,43 @@ public class HConnectionTestingUtility {
       Mockito.spy(new ConnectionImplementation(conf, null, null));
     return connection;
   }
+
+  /**
+   * This coproceesor sleep 2s at first increment/append rpc call.
+   */
+  public static class SleepAtFirstRpcCall extends BaseRegionObserver {
+    static final AtomicLong ct = new AtomicLong(0);
+    static final String SLEEP_TIME_CONF_KEY =
+        "hbase.coprocessor.SleepAtFirstRpcCall.sleepTime";
+    static final long DEFAULT_SLEEP_TIME = 2000;
+    static final AtomicLong sleepTime = new AtomicLong(DEFAULT_SLEEP_TIME);
+
+    public SleepAtFirstRpcCall() {
+    }
+
+    @Override
+    public void postOpen(ObserverContext<RegionCoprocessorEnvironment> c) {
+      RegionCoprocessorEnvironment env = c.getEnvironment();
+      Configuration conf = env.getConfiguration();
+      sleepTime.set(conf.getLong(SLEEP_TIME_CONF_KEY, DEFAULT_SLEEP_TIME));
+    }
+
+    @Override
+    public Result postIncrement(final ObserverContext<RegionCoprocessorEnvironment> e,
+        final Increment increment, final Result result) throws IOException {
+      if (ct.incrementAndGet() == 1) {
+        Threads.sleep(sleepTime.get());
+      }
+      return result;
+    }
+
+    @Override
+    public Result postAppend(final ObserverContext<RegionCoprocessorEnvironment> e,
+        final Append append, final Result result) throws IOException {
+      if (ct.incrementAndGet() == 1) {
+        Threads.sleep(sleepTime.get());
+      }
+      return result;
+    }
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/975f0dd9/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
index ca4b609..bc94b02 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
@@ -31,6 +31,7 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
@@ -169,6 +170,50 @@ public class TestFromClientSide {
   }
 
   /**
+   * Test append result when there are duplicate rpc request.
+   */
+  @Test
+  public void testDuplicateAppend() throws Exception {
+    HTableDescriptor hdt = TEST_UTIL.createTableDescriptor("HCM-testDuplicateAppend");
+    Map<String, String> kvs = new HashMap<String, String>();
+    kvs.put(HConnectionTestingUtility.SleepAtFirstRpcCall.SLEEP_TIME_CONF_KEY, "2000");
+    hdt.addCoprocessor(HConnectionTestingUtility.SleepAtFirstRpcCall.class.getName(), null, 1, kvs);
+    TEST_UTIL.createTable(hdt, new byte[][] { ROW }).close();
+
+    Configuration c = new Configuration(TEST_UTIL.getConfiguration());
+    c.setInt(HConstants.HBASE_CLIENT_PAUSE, 50);
+    // Client will retry beacuse rpc timeout is small than the sleep time of first rpc call
+    c.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, 1500);
+
+    Connection connection = ConnectionFactory.createConnection(c);
+    Table t = connection.getTable(TableName.valueOf("HCM-testDuplicateAppend"));
+    if (t instanceof HTable) {
+      HTable table = (HTable) t;
+      table.setOperationTimeout(3 * 1000);
+
+      try {
+        Append append = new Append(ROW);
+        append.add(TEST_UTIL.fam1, QUALIFIER, VALUE);
+        Result result = table.append(append);
+
+        // Verify expected result
+        Cell[] cells = result.rawCells();
+        assertEquals(1, cells.length);
+        assertKey(cells[0], ROW, TEST_UTIL.fam1, QUALIFIER, VALUE);
+
+        // Verify expected result again
+        Result readResult = table.get(new Get(ROW));
+        cells = readResult.rawCells();
+        assertEquals(1, cells.length);
+        assertKey(cells[0], ROW, TEST_UTIL.fam1, QUALIFIER, VALUE);
+      } finally {
+        table.close();
+        connection.close();
+      }
+    }
+  }
+
+  /**
    * Basic client side validation of HBASE-4536
    */
    @Test

http://git-wip-us.apache.org/repos/asf/hbase/blob/975f0dd9/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncrementsFromClientSide.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncrementsFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncrementsFromClientSide.java
index 6b4ee89..3ddfef4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncrementsFromClientSide.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncrementsFromClientSide.java
@@ -22,6 +22,8 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -29,6 +31,8 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint;
@@ -56,6 +60,7 @@ public class TestIncrementsFromClientSide {
   protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
   private static byte [] ROW = Bytes.toBytes("testRow");
   private static byte [] FAMILY = Bytes.toBytes("testFamily");
+  private static byte [] QUALIFIER = Bytes.toBytes("testQualifier");
   // This test depends on there being only one slave running at at a time. See the @Before
   // method where we do rolling restart.
   protected static int SLAVES = 1;
@@ -79,6 +84,49 @@ public class TestIncrementsFromClientSide {
     TEST_UTIL.shutdownMiniCluster();
   }
 
+  /**
+   * Test increment result when there are duplicate rpc request.
+   */
+  @Test
+  public void testDuplicateIncrement() throws Exception {
+    HTableDescriptor hdt = TEST_UTIL.createTableDescriptor("HCM-testDuplicateIncrement");
+    Map<String, String> kvs = new HashMap<String, String>();
+    kvs.put(HConnectionTestingUtility.SleepAtFirstRpcCall.SLEEP_TIME_CONF_KEY, "2000");
+    hdt.addCoprocessor(HConnectionTestingUtility.SleepAtFirstRpcCall.class.getName(), null, 1, kvs);
+    TEST_UTIL.createTable(hdt, new byte[][] { ROW }).close();
+
+    Configuration c = new Configuration(TEST_UTIL.getConfiguration());
+    c.setInt(HConstants.HBASE_CLIENT_PAUSE, 50);
+    // Client will retry beacuse rpc timeout is small than the sleep time of first rpc call
+    c.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, 1500);
+
+    Connection connection = ConnectionFactory.createConnection(c);
+    Table t = connection.getTable(TableName.valueOf("HCM-testDuplicateIncrement"));
+    if (t instanceof HTable) {
+      HTable table = (HTable) t;
+      table.setOperationTimeout(3 * 1000);
+
+      try {
+        Increment inc = new Increment(ROW);
+        inc.addColumn(TEST_UTIL.fam1, QUALIFIER, 1);
+        Result result = table.increment(inc);
+
+        Cell [] cells = result.rawCells();
+        assertEquals(1, cells.length);
+        assertIncrementKey(cells[0], ROW, TEST_UTIL.fam1, QUALIFIER, 1);
+
+        // Verify expected result
+        Result readResult = table.get(new Get(ROW));
+        cells = readResult.rawCells();
+        assertEquals(1, cells.length);
+        assertIncrementKey(cells[0], ROW, TEST_UTIL.fam1, QUALIFIER, 1);
+      } finally {
+        table.close();
+        connection.close();
+      }
+    }
+  }
+
   @Test
   public void testIncrementWithDeletes() throws Exception {
     LOG.info("Starting " + this.name.getMethodName());

http://git-wip-us.apache.org/repos/asf/hbase/blob/975f0dd9/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java
index 67ac51e..b1ad172 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java
@@ -41,7 +41,6 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Waiter;
 import org.apache.hadoop.hbase.codec.KeyValueCodec;
-import org.apache.hadoop.hbase.exceptions.OperationConflictException;
 import org.apache.hadoop.hbase.testclassification.FlakeyTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -524,16 +523,16 @@ public class TestMultiParallel {
       Increment inc = new Increment(ONE_ROW);
       inc.addColumn(BYTES_FAMILY, QUALIFIER, 1L);
       table.increment(inc);
+
+      // duplicate increment
       inc = new Increment(ONE_ROW);
       inc.addColumn(BYTES_FAMILY, QUALIFIER, 1L);
-      try {
-        table.increment(inc);
-        fail("Should have thrown an exception");
-      } catch (OperationConflictException ex) {
-      }
+      Result result = table.increment(inc);
+      validateResult(result, QUALIFIER, Bytes.toBytes(1L));
+
       Get get = new Get(ONE_ROW);
       get.addColumn(BYTES_FAMILY, QUALIFIER);
-      Result result = table.get(get);
+      result = table.get(get);
       validateResult(result, QUALIFIER, Bytes.toBytes(1L));
 
       // Now run a bunch of requests in parallel, exactly half should succeed.
@@ -561,7 +560,6 @@ public class TestMultiParallel {
             }
             try {
               table.increment(inc);
-            } catch (OperationConflictException ex) { // Some threads are expected to fail.
             } catch (IOException ioEx) {
               fail("Not expected");
             }

http://git-wip-us.apache.org/repos/asf/hbase/blob/975f0dd9/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerHeartbeatMessages.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerHeartbeatMessages.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerHeartbeatMessages.java
index 54bee94..b906e84 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerHeartbeatMessages.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerHeartbeatMessages.java
@@ -489,7 +489,7 @@ public class TestScannerHeartbeatMessages {
     // Instantiate the custom heartbeat region scanners
     @Override
     protected RegionScanner instantiateRegionScanner(Scan scan,
-        List<KeyValueScanner> additionalScanners) throws IOException {
+        List<KeyValueScanner> additionalScanners, long nonceGroup, long nonce) throws IOException {
       if (scan.isReversed()) {
         if (scan.getFilter() != null) {
           scan.getFilter().setReversed(true);


[28/50] [abbrv] hbase git commit: HBASE-16367 Race between master and region server initialization may lead to premature server abort - addendum shortens the waiting duration

Posted by sy...@apache.org.
HBASE-16367 Race between master and region server initialization may lead to premature server abort - addendum shortens the waiting duration


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

Branch: refs/heads/hbase-12439
Commit: 1ecb0fce342ee878cf96f7a3165007192bedb2ef
Parents: 50f3c95
Author: tedyu <yu...@gmail.com>
Authored: Mon Aug 8 10:39:35 2016 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Mon Aug 8 10:39:35 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/regionserver/HRegionServer.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/1ecb0fce/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index ba5d669..0b4ae75 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -819,7 +819,7 @@ public class HRegionServer extends HasThread implements
     blockAndCheckIfStopped(this.clusterStatusTracker);
 
     if (this.initLatch != null) {
-      this.initLatch.await(50, TimeUnit.SECONDS);
+      this.initLatch.await(20, TimeUnit.SECONDS);
     }
     // Retrieve clusterId
     // Since cluster status is now up


[19/50] [abbrv] hbase git commit: Revert "HBASE-16308 Contain protobuf references Gather up the pb references into a few locations only rather than have pb references distributed all about the code base."

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/0206dc67/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 1b3e111..fbd9f51 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
@@ -18,6 +18,12 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.Descriptors;
+import com.google.protobuf.Message;
+import com.google.protobuf.Service;
+import com.google.protobuf.ServiceException;
+
 import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.util.ArrayList;
@@ -37,6 +43,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.HTableDescriptor;
@@ -67,16 +74,6 @@ import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.ReflectionUtils;
 import org.apache.hadoop.hbase.util.Threads;
 
-import com.google.common.annotations.VisibleForTesting;
-
-// DO NOT MAKE USE OF THESE IMPORTS! THEY ARE HERE FOR COPROCESSOR ENDPOINTS ONLY.
-// Internally, we use shaded protobuf. This below are part of our public API.
-import com.google.protobuf.Descriptors;
-import com.google.protobuf.Message;
-import com.google.protobuf.ServiceException;
-import com.google.protobuf.Service;
-// SEE ABOVE NOTE!
-
 /**
  * An implementation of {@link Table}. Used to communicate with a single HBase table.
  * Lightweight. Get as needed and just close when done.
@@ -414,16 +411,23 @@ public class HTable implements Table {
 
     if (get.getConsistency() == Consistency.STRONG) {
       // Good old call.
-      final Get configuredGet = get;
+      final Get getReq = get;
       RegionServerCallable<Result> callable = new RegionServerCallable<Result>(this.connection,
-          this.rpcControllerFactory, getName(), get.getRow()) {
+          getName(), get.getRow()) {
         @Override
-        protected Result call(PayloadCarryingRpcController controller) throws Exception {
-          ClientProtos.GetRequest request = RequestConverter.buildGetRequest(
-              getLocation().getRegionInfo().getRegionName(), configuredGet);
-          ClientProtos.GetResponse response = getStub().get(controller, request);
-          if (response == null) return null;
-          return ProtobufUtil.toResult(response.getResult(), controller.cellScanner());
+        public Result call(int callTimeout) throws IOException {
+          ClientProtos.GetRequest request =
+            RequestConverter.buildGetRequest(getLocation().getRegionInfo().getRegionName(), getReq);
+          PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+          controller.setPriority(tableName);
+          controller.setCallTimeout(callTimeout);
+          try {
+            ClientProtos.GetResponse response = getStub().get(controller, request);
+            if (response == null) return null;
+            return ProtobufUtil.toResult(response.getResult(), controller.cellScanner());
+          } catch (ServiceException se) {
+            throw ProtobufUtil.getRemoteException(se);
+          }
         }
       };
       return rpcCallerFactory.<Result>newCaller(rpcTimeout).callWithRetries(callable,
@@ -439,6 +443,7 @@ public class HTable implements Table {
     return callable.call(operationTimeout);
   }
 
+
   /**
    * {@inheritDoc}
    */
@@ -449,14 +454,16 @@ public class HTable implements Table {
     }
     try {
       Object[] r1 = new Object[gets.size()];
-      batch((List<? extends Row>)gets, r1);
-      // Translate.
+      batch((List) gets, r1);
+
+      // translate.
       Result [] results = new Result[r1.length];
-      int i = 0;
-      for (Object obj: r1) {
-        // Batch ensures if there is a failure we get an exception instead
-        results[i++] = (Result)obj;
+      int i=0;
+      for (Object o : r1) {
+        // batch ensures if there is a failure we get an exception instead
+        results[i++] = (Result) o;
       }
+
       return results;
     } catch (InterruptedException e) {
       throw (InterruptedIOException)new InterruptedIOException().initCause(e);
@@ -504,13 +511,21 @@ public class HTable implements Table {
   public void delete(final Delete delete)
   throws IOException {
     RegionServerCallable<Boolean> callable = new RegionServerCallable<Boolean>(connection,
-        this.rpcControllerFactory, getName(), delete.getRow()) {
+        tableName, delete.getRow()) {
       @Override
-      protected Boolean call(PayloadCarryingRpcController controller) throws Exception {
-        MutateRequest request = RequestConverter.buildMutateRequest(
-          getLocation().getRegionInfo().getRegionName(), delete);
-        MutateResponse response = getStub().mutate(controller, request);
-        return Boolean.valueOf(response.getProcessed());
+      public Boolean call(int callTimeout) throws IOException {
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setPriority(tableName);
+        controller.setCallTimeout(callTimeout);
+
+        try {
+          MutateRequest request = RequestConverter.buildMutateRequest(
+            getLocation().getRegionInfo().getRegionName(), delete);
+          MutateResponse response = getStub().mutate(controller, request);
+          return Boolean.valueOf(response.getProcessed());
+        } catch (ServiceException se) {
+          throw ProtobufUtil.getRemoteException(se);
+        }
       }
     };
     rpcCallerFactory.<Boolean> newCaller(rpcTimeout).callWithRetries(callable,
@@ -566,28 +581,41 @@ public class HTable implements Table {
    */
   @Override
   public void mutateRow(final RowMutations rm) throws IOException {
+    final RetryingTimeTracker tracker = new RetryingTimeTracker();
     PayloadCarryingServerCallable<MultiResponse> callable =
-      new PayloadCarryingServerCallable<MultiResponse>(this.connection, getName(), rm.getRow(),
+      new PayloadCarryingServerCallable<MultiResponse>(connection, getName(), rm.getRow(),
           rpcControllerFactory) {
-      @Override
-      protected MultiResponse call(PayloadCarryingRpcController controller) throws Exception {
-        RegionAction.Builder regionMutationBuilder = RequestConverter.buildRegionAction(
-            getLocation().getRegionInfo().getRegionName(), rm);
-        regionMutationBuilder.setAtomic(true);
-        MultiRequest request =
-            MultiRequest.newBuilder().addRegionAction(regionMutationBuilder.build()).build();
-        ClientProtos.MultiResponse response = getStub().multi(controller, request);
-        ClientProtos.RegionActionResult res = response.getRegionActionResultList().get(0);
-        if (res.hasException()) {
-          Throwable ex = ProtobufUtil.toException(res.getException());
-          if (ex instanceof IOException) {
-            throw (IOException) ex;
+        @Override
+        public MultiResponse call(int callTimeout) throws IOException {
+          tracker.start();
+          controller.setPriority(tableName);
+          int remainingTime = tracker.getRemainingTime(callTimeout);
+          if (remainingTime == 0) {
+            throw new DoNotRetryIOException("Timeout for mutate row");
+          }
+          controller.setCallTimeout(remainingTime);
+          try {
+            RegionAction.Builder regionMutationBuilder = RequestConverter.buildRegionAction(
+                getLocation().getRegionInfo().getRegionName(), rm);
+            regionMutationBuilder.setAtomic(true);
+            MultiRequest request =
+                MultiRequest.newBuilder().addRegionAction(regionMutationBuilder.build()).build();
+            ClientProtos.MultiResponse response = getStub().multi(controller, request);
+            ClientProtos.RegionActionResult res = response.getRegionActionResultList().get(0);
+            if (res.hasException()) {
+              Throwable ex = ProtobufUtil.toException(res.getException());
+              if (ex instanceof IOException) {
+                throw (IOException) ex;
+              }
+              throw new IOException("Failed to mutate row: " +
+                  Bytes.toStringBinary(rm.getRow()), ex);
+            }
+            return ResponseConverter.getResults(request, response, controller.cellScanner());
+          } catch (ServiceException se) {
+            throw ProtobufUtil.getRemoteException(se);
           }
-          throw new IOException("Failed to mutate row: " + Bytes.toStringBinary(rm.getRow()), ex);
         }
-        return ResponseConverter.getResults(request, response, controller.cellScanner());
-      }
-    };
+      };
     AsyncRequestFuture ars = multiAp.submitAll(pool, tableName, rm.getMutations(),
         null, null, callable, operationTimeout);
     ars.waitUntilDone();
@@ -596,31 +624,38 @@ public class HTable implements Table {
     }
   }
 
-  private static void checkHasFamilies(final Mutation mutation) throws IOException {
-    if (mutation.numFamilies() == 0) {
-      throw new IOException("Invalid arguments to " + mutation + ", zero columns specified");
-    }
-  }
-
   /**
    * {@inheritDoc}
    */
   @Override
   public Result append(final Append append) throws IOException {
-    checkHasFamilies(append);
-    RegionServerCallable<Result> callable = new RegionServerCallable<Result>(this.connection,
-        this.rpcControllerFactory, getName(), append.getRow()) {
-      @Override
-      protected Result call(PayloadCarryingRpcController controller) throws Exception {
-        MutateRequest request = RequestConverter.buildMutateRequest(
-          getLocation().getRegionInfo().getRegionName(), append, getNonceGroup(), getNewNonce());
-        MutateResponse response = getStub().mutate(controller, request);
-        if (!response.hasResult()) return null;
-        return ProtobufUtil.toResult(response.getResult(), controller.cellScanner());
-      }
-    };
-    return rpcCallerFactory.<Result> newCaller(this.rpcTimeout).
-        callWithRetries(callable, this.operationTimeout);
+    if (append.numFamilies() == 0) {
+      throw new IOException(
+          "Invalid arguments to append, no columns specified");
+    }
+
+    NonceGenerator ng = this.connection.getNonceGenerator();
+    final long nonceGroup = ng.getNonceGroup(), nonce = ng.newNonce();
+    RegionServerCallable<Result> callable =
+      new RegionServerCallable<Result>(this.connection, getName(), append.getRow()) {
+        @Override
+        public Result call(int callTimeout) throws IOException {
+          PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+          controller.setPriority(getTableName());
+          controller.setCallTimeout(callTimeout);
+          try {
+            MutateRequest request = RequestConverter.buildMutateRequest(
+              getLocation().getRegionInfo().getRegionName(), append, nonceGroup, nonce);
+            MutateResponse response = getStub().mutate(controller, request);
+            if (!response.hasResult()) return null;
+            return ProtobufUtil.toResult(response.getResult(), controller.cellScanner());
+          } catch (ServiceException se) {
+            throw ProtobufUtil.getRemoteException(se);
+          }
+        }
+      };
+    return rpcCallerFactory.<Result> newCaller(rpcTimeout).callWithRetries(callable,
+        this.operationTimeout);
   }
 
   /**
@@ -628,16 +663,27 @@ public class HTable implements Table {
    */
   @Override
   public Result increment(final Increment increment) throws IOException {
-    checkHasFamilies(increment);
+    if (!increment.hasFamilies()) {
+      throw new IOException(
+          "Invalid arguments to increment, no columns specified");
+    }
+    NonceGenerator ng = this.connection.getNonceGenerator();
+    final long nonceGroup = ng.getNonceGroup(), nonce = ng.newNonce();
     RegionServerCallable<Result> callable = new RegionServerCallable<Result>(this.connection,
-        this.rpcControllerFactory, getName(), increment.getRow()) {
+        getName(), increment.getRow()) {
       @Override
-      protected Result call(PayloadCarryingRpcController controller) throws Exception {
-        MutateRequest request = RequestConverter.buildMutateRequest(
-          getLocation().getRegionInfo().getRegionName(), increment, getNonceGroup(), getNewNonce());
-        MutateResponse response = getStub().mutate(controller, request);
-        // Should this check for null like append does?
-        return ProtobufUtil.toResult(response.getResult(), controller.cellScanner());
+      public Result call(int callTimeout) throws IOException {
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setPriority(getTableName());
+        controller.setCallTimeout(callTimeout);
+        try {
+          MutateRequest request = RequestConverter.buildMutateRequest(
+            getLocation().getRegionInfo().getRegionName(), increment, nonceGroup, nonce);
+          MutateResponse response = getStub().mutate(controller, request);
+          return ProtobufUtil.toResult(response.getResult(), controller.cellScanner());
+        } catch (ServiceException se) {
+          throw ProtobufUtil.getRemoteException(se);
+        }
       }
     };
     return rpcCallerFactory.<Result> newCaller(rpcTimeout).callWithRetries(callable,
@@ -676,20 +722,28 @@ public class HTable implements Table {
 
     NonceGenerator ng = this.connection.getNonceGenerator();
     final long nonceGroup = ng.getNonceGroup(), nonce = ng.newNonce();
-    RegionServerCallable<Long> callable = new RegionServerCallable<Long>(this.connection,
-        this.rpcControllerFactory, getName(), row) {
-      @Override
-      protected Long call(PayloadCarryingRpcController controller) throws Exception {
-        MutateRequest request = RequestConverter.buildIncrementRequest(
-          getLocation().getRegionInfo().getRegionName(), row, family,
-          qualifier, amount, durability, nonceGroup, nonce);
-        MutateResponse response = getStub().mutate(controller, request);
-        Result result = ProtobufUtil.toResult(response.getResult(), controller.cellScanner());
-        return Long.valueOf(Bytes.toLong(result.getValue(family, qualifier)));
-      }
-    };
-    return rpcCallerFactory.<Long> newCaller(rpcTimeout).
-        callWithRetries(callable, this.operationTimeout);
+    RegionServerCallable<Long> callable =
+      new RegionServerCallable<Long>(connection, getName(), row) {
+        @Override
+        public Long call(int callTimeout) throws IOException {
+          PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+          controller.setPriority(getTableName());
+          controller.setCallTimeout(callTimeout);
+          try {
+            MutateRequest request = RequestConverter.buildIncrementRequest(
+              getLocation().getRegionInfo().getRegionName(), row, family,
+              qualifier, amount, durability, nonceGroup, nonce);
+            MutateResponse response = getStub().mutate(controller, request);
+            Result result =
+              ProtobufUtil.toResult(response.getResult(), controller.cellScanner());
+            return Long.valueOf(Bytes.toLong(result.getValue(family, qualifier)));
+          } catch (ServiceException se) {
+            throw ProtobufUtil.getRemoteException(se);
+          }
+        }
+      };
+    return rpcCallerFactory.<Long> newCaller(rpcTimeout).callWithRetries(callable,
+        this.operationTimeout);
   }
 
   /**
@@ -700,19 +754,26 @@ public class HTable implements Table {
       final byte [] family, final byte [] qualifier, final byte [] value,
       final Put put)
   throws IOException {
-    RegionServerCallable<Boolean> callable = new RegionServerCallable<Boolean>(this.connection,
-        this.rpcControllerFactory, getName(), row) {
-      @Override
-      protected Boolean call(PayloadCarryingRpcController controller) throws Exception {
-        MutateRequest request = RequestConverter.buildMutateRequest(
-          getLocation().getRegionInfo().getRegionName(), row, family, qualifier,
-          new BinaryComparator(value), CompareType.EQUAL, put);
-        MutateResponse response = getStub().mutate(controller, request);
-        return Boolean.valueOf(response.getProcessed());
-      }
-    };
-    return rpcCallerFactory.<Boolean> newCaller(rpcTimeout).
-        callWithRetries(callable, this.operationTimeout);
+    RegionServerCallable<Boolean> callable =
+      new RegionServerCallable<Boolean>(connection, getName(), row) {
+        @Override
+        public Boolean call(int callTimeout) throws IOException {
+          PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+          controller.setPriority(tableName);
+          controller.setCallTimeout(callTimeout);
+          try {
+            MutateRequest request = RequestConverter.buildMutateRequest(
+                getLocation().getRegionInfo().getRegionName(), row, family, qualifier,
+                new BinaryComparator(value), CompareType.EQUAL, put);
+            MutateResponse response = getStub().mutate(controller, request);
+            return Boolean.valueOf(response.getProcessed());
+          } catch (ServiceException se) {
+            throw ProtobufUtil.getRemoteException(se);
+          }
+        }
+      };
+    return rpcCallerFactory.<Boolean> newCaller(rpcTimeout).callWithRetries(callable,
+        this.operationTimeout);
   }
 
   /**
@@ -723,42 +784,57 @@ public class HTable implements Table {
       final byte [] qualifier, final CompareOp compareOp, final byte [] value,
       final Put put)
   throws IOException {
-    RegionServerCallable<Boolean> callable = new RegionServerCallable<Boolean>(this.connection,
-        this.rpcControllerFactory, getName(), row) {
-      @Override
-      protected Boolean call(PayloadCarryingRpcController controller) throws Exception {
-        CompareType compareType = CompareType.valueOf(compareOp.name());
-        MutateRequest request = RequestConverter.buildMutateRequest(
-          getLocation().getRegionInfo().getRegionName(), row, family, qualifier,
-          new BinaryComparator(value), compareType, put);
-        MutateResponse response = getStub().mutate(controller, request);
-        return Boolean.valueOf(response.getProcessed());
-      }
-    };
-    return rpcCallerFactory.<Boolean> newCaller(rpcTimeout).
-        callWithRetries(callable, this.operationTimeout);
+    RegionServerCallable<Boolean> callable =
+      new RegionServerCallable<Boolean>(connection, getName(), row) {
+        @Override
+        public Boolean call(int callTimeout) throws IOException {
+          PayloadCarryingRpcController controller = new PayloadCarryingRpcController();
+          controller.setPriority(tableName);
+          controller.setCallTimeout(callTimeout);
+          try {
+            CompareType compareType = CompareType.valueOf(compareOp.name());
+            MutateRequest request = RequestConverter.buildMutateRequest(
+              getLocation().getRegionInfo().getRegionName(), row, family, qualifier,
+                new BinaryComparator(value), compareType, put);
+            MutateResponse response = getStub().mutate(controller, request);
+            return Boolean.valueOf(response.getProcessed());
+          } catch (ServiceException se) {
+            throw ProtobufUtil.getRemoteException(se);
+          }
+        }
+      };
+    return rpcCallerFactory.<Boolean> newCaller(rpcTimeout).callWithRetries(callable,
+        this.operationTimeout);
   }
 
   /**
    * {@inheritDoc}
    */
   @Override
-  public boolean checkAndDelete(final byte [] row, final byte [] family, final byte [] qualifier,
-      final byte [] value, final Delete delete)
+  public boolean checkAndDelete(final byte [] row,
+      final byte [] family, final byte [] qualifier, final byte [] value,
+      final Delete delete)
   throws IOException {
-    RegionServerCallable<Boolean> callable = new RegionServerCallable<Boolean>(this.connection,
-        this.rpcControllerFactory, getName(), row) {
-      @Override
-      protected Boolean call(PayloadCarryingRpcController controller) throws Exception {
-        MutateRequest request = RequestConverter.buildMutateRequest(
-          getLocation().getRegionInfo().getRegionName(), row, family, qualifier,
-          new BinaryComparator(value), CompareType.EQUAL, delete);
-        MutateResponse response = getStub().mutate(controller, request);
-        return Boolean.valueOf(response.getProcessed());
-      }
-    };
-    return rpcCallerFactory.<Boolean> newCaller(rpcTimeout).
-        callWithRetries(callable, this.operationTimeout);
+    RegionServerCallable<Boolean> callable =
+      new RegionServerCallable<Boolean>(connection, getName(), row) {
+        @Override
+        public Boolean call(int callTimeout) throws IOException {
+          PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+          controller.setPriority(tableName);
+          controller.setCallTimeout(callTimeout);
+          try {
+            MutateRequest request = RequestConverter.buildMutateRequest(
+              getLocation().getRegionInfo().getRegionName(), row, family, qualifier,
+                new BinaryComparator(value), CompareType.EQUAL, delete);
+            MutateResponse response = getStub().mutate(controller, request);
+            return Boolean.valueOf(response.getProcessed());
+          } catch (ServiceException se) {
+            throw ProtobufUtil.getRemoteException(se);
+          }
+        }
+      };
+    return rpcCallerFactory.<Boolean> newCaller(rpcTimeout).callWithRetries(callable,
+        this.operationTimeout);
   }
 
   /**
@@ -769,18 +845,25 @@ public class HTable implements Table {
       final byte [] qualifier, final CompareOp compareOp, final byte [] value,
       final Delete delete)
   throws IOException {
-    RegionServerCallable<Boolean> callable = new RegionServerCallable<Boolean>(this.connection,
-        this.rpcControllerFactory, getName(), row) {
-      @Override
-      protected Boolean call(PayloadCarryingRpcController controller) throws Exception {
-        CompareType compareType = CompareType.valueOf(compareOp.name());
-        MutateRequest request = RequestConverter.buildMutateRequest(
-          getLocation().getRegionInfo().getRegionName(), row, family, qualifier,
-          new BinaryComparator(value), compareType, delete);
-        MutateResponse response = getStub().mutate(controller, request);
-        return Boolean.valueOf(response.getProcessed());
-      }
-    };
+    RegionServerCallable<Boolean> callable =
+      new RegionServerCallable<Boolean>(connection, getName(), row) {
+        @Override
+        public Boolean call(int callTimeout) throws IOException {
+          PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+          controller.setPriority(tableName);
+          controller.setCallTimeout(callTimeout);
+          try {
+            CompareType compareType = CompareType.valueOf(compareOp.name());
+            MutateRequest request = RequestConverter.buildMutateRequest(
+              getLocation().getRegionInfo().getRegionName(), row, family, qualifier,
+                new BinaryComparator(value), compareType, delete);
+            MutateResponse response = getStub().mutate(controller, request);
+            return Boolean.valueOf(response.getProcessed());
+          } catch (ServiceException se) {
+            throw ProtobufUtil.getRemoteException(se);
+          }
+        }
+      };
     return rpcCallerFactory.<Boolean> newCaller(rpcTimeout).callWithRetries(callable,
         this.operationTimeout);
   }
@@ -792,28 +875,40 @@ public class HTable implements Table {
   public boolean checkAndMutate(final byte [] row, final byte [] family, final byte [] qualifier,
     final CompareOp compareOp, final byte [] value, final RowMutations rm)
     throws IOException {
+    final RetryingTimeTracker tracker = new RetryingTimeTracker();
     PayloadCarryingServerCallable<MultiResponse> callable =
       new PayloadCarryingServerCallable<MultiResponse>(connection, getName(), rm.getRow(),
         rpcControllerFactory) {
         @Override
-        protected MultiResponse call(PayloadCarryingRpcController controller) throws Exception {
-          CompareType compareType = CompareType.valueOf(compareOp.name());
-          MultiRequest request = RequestConverter.buildMutateRequest(
-            getLocation().getRegionInfo().getRegionName(), row, family, qualifier,
-            new BinaryComparator(value), compareType, rm);
-          ClientProtos.MultiResponse response = getStub().multi(controller, request);
-          ClientProtos.RegionActionResult res = response.getRegionActionResultList().get(0);
-          if (res.hasException()) {
-            Throwable ex = ProtobufUtil.toException(res.getException());
-            if (ex instanceof IOException) {
-              throw (IOException)ex;
+        public MultiResponse call(int callTimeout) throws IOException {
+          tracker.start();
+          controller.setPriority(tableName);
+          int remainingTime = tracker.getRemainingTime(callTimeout);
+          if (remainingTime == 0) {
+            throw new DoNotRetryIOException("Timeout for mutate row");
+          }
+          controller.setCallTimeout(remainingTime);
+          try {
+            CompareType compareType = CompareType.valueOf(compareOp.name());
+            MultiRequest request = RequestConverter.buildMutateRequest(
+              getLocation().getRegionInfo().getRegionName(), row, family, qualifier,
+              new BinaryComparator(value), compareType, rm);
+            ClientProtos.MultiResponse response = getStub().multi(controller, request);
+            ClientProtos.RegionActionResult res = response.getRegionActionResultList().get(0);
+            if (res.hasException()) {
+              Throwable ex = ProtobufUtil.toException(res.getException());
+              if(ex instanceof IOException) {
+                throw (IOException)ex;
+              }
+              throw new IOException("Failed to checkAndMutate row: "+
+                                    Bytes.toStringBinary(rm.getRow()), ex);
             }
-            throw new IOException("Failed to checkAndMutate row: "+ Bytes.toStringBinary(rm.getRow()), ex);
+            return ResponseConverter.getResults(request, response, controller.cellScanner());
+          } catch (ServiceException se) {
+            throw ProtobufUtil.getRemoteException(se);
           }
-          return ResponseConverter.getResults(request, response, controller.cellScanner());
         }
       };
-
     /**
      *  Currently, we use one array to store 'processed' flag which is returned by server.
      *  It is excessive to send such a large array, but that is required by the framework right now
@@ -873,6 +968,7 @@ public class HTable implements Table {
   }
 
   /**
+   * {@inheritDoc}
    * @throws IOException
    */
   void flushCommits() throws IOException {
@@ -1049,18 +1145,19 @@ public class HTable implements Table {
     for (final byte[] r : keys) {
       final RegionCoprocessorRpcChannel channel =
           new RegionCoprocessorRpcChannel(connection, tableName, r);
-      Future<R> future = pool.submit(new Callable<R>() {
-        @Override
-        public R call() throws Exception {
-          T instance = ProtobufUtil.newServiceStub(service, channel);
-          R result = callable.call(instance);
-          byte[] region = channel.getLastRegion();
-          if (callback != null) {
-            callback.update(region, r, result);
-          }
-          return result;
-        }
-      });
+      Future<R> future = pool.submit(
+          new Callable<R>() {
+            @Override
+            public R call() throws Exception {
+              T instance = ProtobufUtil.newServiceStub(service, channel);
+              R result = callable.call(instance);
+              byte[] region = channel.getLastRegion();
+              if (callback != null) {
+                callback.update(region, r, result);
+              }
+              return result;
+            }
+          });
       futures.put(r, future);
     }
     for (Map.Entry<byte[],Future<R>> e : futures.entrySet()) {
@@ -1113,6 +1210,9 @@ public class HTable implements Table {
     return tableName + ";" + connection;
   }
 
+  /**
+   * {@inheritDoc}
+   */
   @Override
   public <R extends Message> Map<byte[], R> batchCoprocessorService(
       Descriptors.MethodDescriptor methodDescriptor, Message request,
@@ -1121,13 +1221,14 @@ public class HTable implements Table {
         Bytes.BYTES_COMPARATOR));
     batchCoprocessorService(methodDescriptor, request, startKey, endKey, responsePrototype,
         new Callback<R>() {
-      @Override
-      public void update(byte[] region, byte[] row, R result) {
-        if (region != null) {
-          results.put(region, result);
-        }
-      }
-    });
+
+          @Override
+          public void update(byte[] region, byte[] row, R result) {
+            if (region != null) {
+              results.put(region, result);
+            }
+          }
+        });
     return results;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/0206dc67/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterCallable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterCallable.java
index ae62255..66d3c21 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterCallable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterCallable.java
@@ -21,24 +21,16 @@ package org.apache.hadoop.hbase.client;
 import java.io.Closeable;
 import java.io.IOException;
 
-import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
-import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
-import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
-
 /**
  * A RetryingCallable for master operations.
  * @param <V> return type
  */
-// Like RegionServerCallable
 abstract class MasterCallable<V> implements RetryingCallable<V>, Closeable {
   protected ClusterConnection connection;
   protected MasterKeepAliveConnection master;
-  private final PayloadCarryingRpcController rpcController;
 
-  MasterCallable(final Connection connection,
-      final RpcControllerFactory rpcConnectionFactory) {
+  public MasterCallable(final Connection connection) {
     this.connection = (ClusterConnection) connection;
-    this.rpcController = rpcConnectionFactory.newController();
   }
 
   @Override
@@ -67,31 +59,4 @@ abstract class MasterCallable<V> implements RetryingCallable<V>, Closeable {
   public long sleep(long pause, int tries) {
     return ConnectionUtils.getPauseTime(pause, tries);
   }
-
-  /**
-   * Override that changes Exception from {@link Exception} to {@link IOException}. It also does
-   * setup of an rpcController and calls through to the unimplemented
-   * call(PayloadCarryingRpcController) method; implement this method to add your rpc invocation.
-   */
-  @Override
-  // Same trick as in RegionServerCallable so users don't have to copy/paste so much boilerplate
-  // and so we contain references to protobuf. We can't set priority on the rpcController as
-  // we do in RegionServerCallable because we don't always have a Table when we call.
-  public V call(int callTimeout) throws IOException {
-    try {
-      this.rpcController.setCallTimeout(callTimeout);
-      return call(this.rpcController);
-    } catch (Exception e) {
-      throw ProtobufUtil.handleRemoteException(e);
-    }
-  }
-
-  /**
-   * Run RPC call.
-   * @param rpcController PayloadCarryingRpcController is a mouthful but it at a minimum is a
-   * facade on protobuf so we don't have to put protobuf everywhere; we can keep it behind this
-   * class.
-   * @throws Exception
-   */
-  protected abstract V call(PayloadCarryingRpcController rpcController) throws Exception;
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/0206dc67/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterKeepAliveConnection.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterKeepAliveConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterKeepAliveConnection.java
index 47693f4..e445b78 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterKeepAliveConnection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterKeepAliveConnection.java
@@ -33,7 +33,8 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos;
  * against the master on the MasterProtos.MasterService.BlockingInterface; but not by
  * final user code. Hence it's package protected.
  */
-interface MasterKeepAliveConnection extends MasterProtos.MasterService.BlockingInterface {
+interface MasterKeepAliveConnection
+extends MasterProtos.MasterService.BlockingInterface {
   // Do this instead of implement Closeable because closeable returning IOE is PITA.
   void close();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/0206dc67/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java
index a3162f4..e764ceb 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java
@@ -30,9 +30,8 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.protobuf.ResponseConverter;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
@@ -42,14 +41,14 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.ServiceException;
 
 /**
  * Callable that handles the <code>multi</code> method call going against a single
- * regionserver; i.e. A RegionServerCallable for the multi call (It is NOT a
- * RegionServerCallable that goes against multiple regions).
+ * regionserver; i.e. A {@link RegionServerCallable} for the multi call (It is not a
+ * {@link RegionServerCallable} that goes against multiple regions.
  * @param <R>
  */
-@InterfaceAudience.Private
 class MultiServerCallable<R> extends PayloadCarryingServerCallable<MultiResponse> {
   private final MultiAction<R> multiAction;
   private final boolean cellBlock;
@@ -80,7 +79,7 @@ class MultiServerCallable<R> extends PayloadCarryingServerCallable<MultiResponse
   }
 
   @Override
-  protected MultiResponse call(PayloadCarryingRpcController controller) throws Exception {
+  public MultiResponse call(int callTimeout) throws IOException {
     int countOfActions = this.multiAction.size();
     if (countOfActions <= 0) throw new DoNotRetryIOException("No Actions");
     MultiRequest.Builder multiRequestBuilder = MultiRequest.newBuilder();
@@ -99,8 +98,10 @@ class MultiServerCallable<R> extends PayloadCarryingServerCallable<MultiResponse
       regionActionBuilder.clear();
       regionActionBuilder.setRegion(RequestConverter.buildRegionSpecifier(
           HBaseProtos.RegionSpecifier.RegionSpecifierType.REGION_NAME, regionName));
+
+
       if (this.cellBlock) {
-        // Pre-size. Presume at least a KV per Action.  There are likely more.
+        // Presize.  Presume at least a KV per Action.  There are likely more.
         if (cells == null) cells = new ArrayList<CellScannable>(countOfActions);
         // Send data in cellblocks. The call to buildNoDataMultiRequest will skip RowMutations.
         // They have already been handled above. Guess at count of cells
@@ -115,18 +116,18 @@ class MultiServerCallable<R> extends PayloadCarryingServerCallable<MultiResponse
 
     // Controller optionally carries cell data over the proxy/service boundary and also
     // optionally ferries cell response data back out again.
-    PayloadCarryingRpcController payloadCarryingRpcController = null;
-    if (cells != null) {
-      // Cast. Will fail if we have been passed wrong RpcController type.
-      payloadCarryingRpcController = (PayloadCarryingRpcController)controller;
-      payloadCarryingRpcController.setCellScanner(CellUtil.createCellScanner(cells));
-    }
+    if (cells != null) controller.setCellScanner(CellUtil.createCellScanner(cells));
+    controller.setPriority(getTableName());
+    controller.setCallTimeout(callTimeout);
     ClientProtos.MultiResponse responseProto;
     ClientProtos.MultiRequest requestProto = multiRequestBuilder.build();
-    responseProto = getStub().multi(controller, requestProto);
+    try {
+      responseProto = getStub().multi(controller, requestProto);
+    } catch (ServiceException e) {
+      throw ProtobufUtil.getRemoteException(e);
+    }
     if (responseProto == null) return null; // Occurs on cancel
-    return ResponseConverter.getResults(requestProto, responseProto,
-        payloadCarryingRpcController ==  null? null: payloadCarryingRpcController.cellScanner());
+    return ResponseConverter.getResults(requestProto, responseProto, controller.cellScanner());
   }
 
   /**
@@ -150,4 +151,4 @@ class MultiServerCallable<R> extends PayloadCarryingServerCallable<MultiResponse
   ServerName getServerName() {
     return location.getServerName();
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0206dc67/hbase-client/src/main/java/org/apache/hadoop/hbase/client/PayloadCarryingServerCallable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/PayloadCarryingServerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/PayloadCarryingServerCallable.java
index 83d857b..d94f069 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/PayloadCarryingServerCallable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/PayloadCarryingServerCallable.java
@@ -16,51 +16,33 @@
  */
 package org.apache.hadoop.hbase.client;
 
-import java.io.IOException;
-
-import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 
 /**
- * This class is used to unify HTable calls with AsyncProcess Framework. HTable can use
- * AsyncProcess directly though this class. Also adds global timeout tracking on top of
- * RegionServerCallable and implements Cancellable.
+ * This class is used to unify HTable calls with AsyncProcess Framework.
+ * HTable can use AsyncProcess directly though this class.
  */
 @InterfaceAudience.Private
-abstract class PayloadCarryingServerCallable<T> extends RegionServerCallable<T>
-    implements Cancellable {
-  private final RetryingTimeTracker tracker = new RetryingTimeTracker();
-
-  PayloadCarryingServerCallable(Connection connection, TableName tableName, byte[] row,
-      RpcControllerFactory rpcControllerFactory) {
-    super(connection, rpcControllerFactory, tableName, row);
-  }
+public abstract class PayloadCarryingServerCallable<T>
+    extends RegionServerCallable<T> implements Cancellable {
+  protected PayloadCarryingRpcController controller;
 
-  /* Override so can mess with the callTimeout.
-   * (non-Javadoc)
-   * @see org.apache.hadoop.hbase.client.RegionServerCallable#rpcCall(int)
-   */
-  @Override
-  public T call(int callTimeout) throws IOException {
-    // It is expected (it seems) that tracker.start can be called multiple times (on each trip
-    // through the call when retrying). Also, we can call start and no need of a stop.
-    this.tracker.start();
-    int remainingTime = tracker.getRemainingTime(callTimeout);
-    if (remainingTime == 0) {
-      throw new DoNotRetryIOException("Timeout for mutate row");
-    }
-    return super.call(remainingTime);
+  public PayloadCarryingServerCallable(Connection connection, TableName tableName, byte[] row,
+    RpcControllerFactory rpcControllerFactory) {
+    super(connection, tableName, row);
+    this.controller = rpcControllerFactory.newController();
   }
 
   @Override
   public void cancel() {
-    getRpcController().startCancel();
+    controller.startCancel();
   }
 
   @Override
   public boolean isCancelled() {
-    return getRpcController().isCanceled();
+    return controller.isCanceled();
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0206dc67/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionAdminServiceCallable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionAdminServiceCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionAdminServiceCallable.java
index 4e347dd..54c93a0 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionAdminServiceCallable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionAdminServiceCallable.java
@@ -27,30 +27,31 @@ import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.RegionLocations;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
-import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
 import org.apache.hadoop.hbase.util.Bytes;
 
 /**
- * Similar to RegionServerCallable but for the AdminService interface. This service callable
+ * Similar to {@link RegionServerCallable} but for the AdminService interface. This service callable
  * assumes a Table and row and thus does region locating similar to RegionServerCallable.
- * Works against Admin stub rather than Client stub.
  */
 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="URF_UNREAD_PUBLIC_OR_PROTECTED_FIELD",
   justification="stub used by ipc")
 @InterfaceAudience.Private
 public abstract class RegionAdminServiceCallable<T> implements RetryingCallable<T> {
-  protected AdminService.BlockingInterface stub;
-  protected final RpcControllerFactory rpcControllerFactory;
-  private PayloadCarryingRpcController controller = null;
 
   protected final ClusterConnection connection;
+
+  protected final RpcControllerFactory rpcControllerFactory;
+
+  protected AdminService.BlockingInterface stub;
+
   protected HRegionLocation location;
+
   protected final TableName tableName;
   protected final byte[] row;
   protected final int replicaId;
+
   protected final static int MIN_WAIT_DEAD_SERVER = 10000;
 
   public RegionAdminServiceCallable(ClusterConnection connection,
@@ -81,13 +82,16 @@ public abstract class RegionAdminServiceCallable<T> implements RetryingCallable<
     if (Thread.interrupted()) {
       throw new InterruptedIOException();
     }
+
     if (reload || location == null) {
       location = getLocation(!reload);
     }
+
     if (location == null) {
       // With this exception, there will be a retry.
       throw new HBaseIOException(getExceptionMessage());
     }
+
     this.setStub(connection.getAdmin(location.getServerName()));
   }
 
@@ -163,39 +167,7 @@ public abstract class RegionAdminServiceCallable<T> implements RetryingCallable<
     if (rl == null) {
       throw new RetriesExhaustedException("Can't get the locations");
     }
-    return rl;
-  }
-
-  /**
-   * Override that changes Exception from {@link Exception} to {@link IOException}. It also does
-   * setup of an rpcController and calls through to the unimplemented
-   * call(PayloadCarryingRpcController) method; implement this method to add your rpc invocation.
-   */
-  @Override
-  // Same trick as in RegionServerCallable so users don't have to copy/paste so much boilerplate
-  // and so we contain references to protobuf. We can't set priority on the rpcController as
-  // we do in RegionServerCallable because we don't always have a Table when we call.
-  public T call(int callTimeout) throws IOException {
-    this.controller = rpcControllerFactory.newController();
-    this.controller.setPriority(this.tableName);
-    this.controller.setCallTimeout(callTimeout);
-    try {
-      return call(this.controller);
-    } catch (Exception e) {
-      throw ProtobufUtil.handleRemoteException(e);
-    }
-  }
 
-  PayloadCarryingRpcController getCurrentPayloadCarryingRpcController() {
-    return this.controller;
+    return rl;
   }
-
-  /**
-   * Run RPC call.
-   * @param rpcController PayloadCarryingRpcController is a mouthful but it at a minimum is a
-   * facade on protobuf so we don't have to put protobuf everywhere; we can keep it behind this
-   * class.
-   * @throws Exception
-   */
-  protected abstract T call(PayloadCarryingRpcController rpcController) throws Exception;
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0206dc67/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionServerCallable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionServerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionServerCallable.java
index 861b375..d878bae 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionServerCallable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionServerCallable.java
@@ -1,4 +1,5 @@
 /**
+ *
  * 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
@@ -23,20 +24,12 @@ import java.io.IOException;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
-import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
-import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
 
 /**
- * Implementations make an rpc call against a RegionService via a protobuf Service.
- * Implement #rpcCall(RpcController) and then call {@link #call(int)} to
- * trigger the rpc. The {@link #call(int)} eventually invokes your
- * #rpcCall(RpcController) meanwhile saving you having to write a bunch of
- * boilerplate. The {@link #call(int)} implementation is from {@link RpcRetryingCaller} so rpcs are
- * retried on fail.
- *
- * <p>TODO: this class is actually tied to one region, because most of the paths make use of
+ * Implementations call a RegionServer and implement {@link #call(int)}.
+ * Passed to a {@link RpcRetryingCaller} so we retry on fail.
+ * TODO: this class is actually tied to one region, because most of the paths make use of
  *       the regioninfo part of location when building requests. The only reason it works for
  *       multi-region requests (e.g. batch) is that they happen to not use the region parts.
  *       This could be done cleaner (e.g. having a generic parameter and 2 derived classes,
@@ -44,27 +37,18 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
  * @param <T> the class that the ServerCallable handles
  */
 @InterfaceAudience.Private
-public abstract class RegionServerCallable<T> extends AbstractRegionServerCallable<T> {
+public abstract class RegionServerCallable<T> extends AbstractRegionServerCallable<T> implements
+    RetryingCallable<T> {
+
   private ClientService.BlockingInterface stub;
-  private final PayloadCarryingRpcController rpcController;
 
   /**
    * @param connection Connection to use.
    * @param tableName Table name to which <code>row</code> belongs.
    * @param row The row we want in <code>tableName</code>.
    */
-  public RegionServerCallable(Connection connection, RpcControllerFactory rpcControllerFactory,
-      TableName tableName, byte [] row) {
-    this(connection, rpcControllerFactory.newController(), tableName, row);
-  }
-
-  public RegionServerCallable(Connection connection, PayloadCarryingRpcController rpcController,
-      TableName tableName, byte [] row) {
+  public RegionServerCallable(Connection connection, TableName tableName, byte [] row) {
     super(connection, tableName, row);
-    this.rpcController = rpcController;
-    if (this.rpcController != null) {
-      this.rpcController.setPriority(tableName);
-    }
   }
 
   void setClientByServiceName(ServerName service) throws IOException {
@@ -85,42 +69,4 @@ public abstract class RegionServerCallable<T> extends AbstractRegionServerCallab
   void setStub(final ClientService.BlockingInterface stub) {
     this.stub = stub;
   }
-
-  /**
-   * Override that changes Exception from {@link Exception} to {@link IOException}. It also does
-   * setup of an rpcController and calls through to the unimplemented
-   * call(PayloadCarryingRpcController) method; implement this method to add your rpc invocation.
-   */
-  @Override
-  public T call(int callTimeout) throws IOException {
-    if (this.rpcController != null) {
-      this.rpcController.setCallTimeout(callTimeout);
-    }
-    try {
-      return call(this.rpcController);
-    } catch (Exception e) {
-      throw ProtobufUtil.handleRemoteException(e);
-    }
-  }
-
-  /**
-   * Run RPC call.
-   * @param rpcController PayloadCarryingRpcController is a mouthful but it at a minimum is a
-   * facade on protobuf so we don't have to put protobuf everywhere; we can keep it behind this
-   * class.
-   * @throws Exception
-   */
-  protected abstract T call(PayloadCarryingRpcController rpcController) throws Exception;
-
-  public PayloadCarryingRpcController getRpcController() {
-    return this.rpcController;
-  }
-
-  long getNonceGroup() {
-    return getConnection().getNonceGenerator().getNonceGroup();
-  }
-
-  long getNewNonce() {
-    return getConnection().getNonceGenerator().newNonce();
-  }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0206dc67/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetryingTimeTracker.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetryingTimeTracker.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetryingTimeTracker.java
index b9438e6..24288e6 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetryingTimeTracker.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetryingTimeTracker.java
@@ -22,6 +22,7 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
  * Tracks the amount of time remaining for an operation.
  */
 class RetryingTimeTracker {
+
   private long globalStartTime = -1;
 
   public void start() {
@@ -37,19 +38,16 @@ class RetryingTimeTracker {
       if (callTimeout == Integer.MAX_VALUE) {
         return Integer.MAX_VALUE;
       }
-      long remaining = EnvironmentEdgeManager.currentTime() - this.globalStartTime;
-      long remainingTime = callTimeout - remaining;
+      int remainingTime = (int) (
+        callTimeout -
+        (EnvironmentEdgeManager.currentTime() - this.globalStartTime));
       if (remainingTime < 1) {
         // If there is no time left, we're trying anyway. It's too late.
         // 0 means no timeout, and it's not the intent here. So we secure both cases by
         // resetting to the minimum.
         remainingTime = 1;
       }
-      if (remainingTime > Integer.MAX_VALUE) {
-        throw new RuntimeException("remainingTime=" + remainingTime +
-            " which is > Integer.MAX_VALUE");
-      }
-      return (int)remainingTime;
+      return remainingTime;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/0206dc67/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedScannerCallable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedScannerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedScannerCallable.java
index 644337d..0c2d345 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedScannerCallable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedScannerCallable.java
@@ -176,9 +176,9 @@ public class ReversedScannerCallable extends ScannerCallable {
 
   @Override
   public ScannerCallable getScannerCallableForReplica(int id) {
-    ReversedScannerCallable r = new ReversedScannerCallable(getConnection(), this.tableName,
+    ReversedScannerCallable r = new ReversedScannerCallable(this.cConnection, this.tableName,
         this.getScan(), this.scanMetrics, this.locateStartRow, controllerFactory, id);
     r.setCaching(this.getCaching());
     return r;
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0206dc67/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallable.java
deleted file mode 100644
index 68a4aa2..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallable.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/**
- * 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
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.client;
-
-import java.io.Closeable;
-import java.io.IOException;
-
-import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
-
-/**
- * A RetryingCallable for RPC connection operations.
- * @param <V> return type
- */
-abstract class RpcRetryingCallable<V> implements RetryingCallable<V>, Closeable {
-  @Override
-  public void prepare(boolean reload) throws IOException {
-  }
-
-  @Override
-  public void close() throws IOException {
-  }
-
-  @Override
-  public void throwable(Throwable t, boolean retrying) {
-  }
-
-  @Override
-  public String getExceptionMessageAdditionalDetail() {
-    return "";
-  }
-
-  @Override
-  public long sleep(long pause, int tries) {
-    return ConnectionUtils.getPauseTime(pause, tries);
-  }
-
-  @Override
-  // Same trick as in RegionServerCallable so users don't have to copy/paste so much boilerplate
-  // and so we contain references to protobuf.
-  public V call(int callTimeout) throws IOException {
-    try {
-      return rpcCall(callTimeout);
-    } catch (Exception e) {
-      throw ProtobufUtil.handleRemoteException(e);
-    }
-  }
-
-  protected abstract V rpcCall(int callTimeout) throws Exception;
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/0206dc67/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 2b2e4c8..b4cd2ef 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
@@ -22,6 +22,9 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 import java.io.IOException;
 
+/**
+ *
+ */
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
 public interface RpcRetryingCaller<T> {
@@ -49,4 +52,4 @@ public interface RpcRetryingCaller<T> {
    */
   T callWithoutRetries(RetryingCallable<T> callable, int callTimeout)
   throws IOException, RuntimeException;
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0206dc67/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 f92aeae..1c723c5 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
@@ -36,7 +36,6 @@ public class RpcRetryingCallerFactory {
   private final int rpcTimeout;
   private final RetryingCallerInterceptor interceptor;
   private final int startLogErrorsCnt;
-  /* These below data members are UNUSED!!!*/
   private final boolean enableBackPressure;
   private ServerStatisticTracker stats;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/0206dc67/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerWithReadReplicas.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerWithReadReplicas.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerWithReadReplicas.java
index 2785648..65dbb10 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerWithReadReplicas.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerWithReadReplicas.java
@@ -29,6 +29,8 @@ import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseIOException;
@@ -44,6 +46,8 @@ import org.apache.hadoop.hbase.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 
+import com.google.protobuf.ServiceException;
+
 
 /**
  * Caller that goes to replica if the primary region does no answer within a configurable
@@ -53,6 +57,8 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
  */
 @InterfaceAudience.Private
 public class RpcRetryingCallerWithReadReplicas {
+  private static final Log LOG = LogFactory.getLog(RpcRetryingCallerWithReadReplicas.class);
+
   protected final ExecutorService pool;
   protected final ClusterConnection cConnection;
   protected final Configuration conf;
@@ -92,7 +98,7 @@ public class RpcRetryingCallerWithReadReplicas {
     private final PayloadCarryingRpcController controller;
 
     public ReplicaRegionServerCallable(int id, HRegionLocation location) {
-      super(RpcRetryingCallerWithReadReplicas.this.cConnection, rpcControllerFactory,
+      super(RpcRetryingCallerWithReadReplicas.this.cConnection,
           RpcRetryingCallerWithReadReplicas.this.tableName, get.getRow());
       this.id = id;
       this.location = location;
@@ -135,20 +141,28 @@ public class RpcRetryingCallerWithReadReplicas {
     }
 
     @Override
-    protected Result call(PayloadCarryingRpcController controller) throws Exception {
+    public Result call(int callTimeout) throws Exception {
       if (controller.isCanceled()) return null;
+
       if (Thread.interrupted()) {
         throw new InterruptedIOException();
       }
+
       byte[] reg = location.getRegionInfo().getRegionName();
+
       ClientProtos.GetRequest request =
           RequestConverter.buildGetRequest(reg, get);
       controller.setCallTimeout(callTimeout);
-      ClientProtos.GetResponse response = getStub().get(controller, request);
-      if (response == null) {
-        return null;
+
+      try {
+        ClientProtos.GetResponse response = getStub().get(controller, request);
+        if (response == null) {
+          return null;
+        }
+        return ProtobufUtil.toResult(response.getResult(), controller.cellScanner());
+      } catch (ServiceException se) {
+        throw ProtobufUtil.getRemoteException(se);
       }
-      return ProtobufUtil.toResult(response.getResult(), controller.cellScanner());
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/0206dc67/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java
index 1689d11..72d69ec 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java
@@ -52,6 +52,9 @@ import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.net.DNS;
 
+import com.google.protobuf.ServiceException;
+import com.google.protobuf.TextFormat;
+
 /**
  * Scanner operations such as create, next, etc.
  * Used by {@link ResultScanner}s made by {@link Table}. Passed to a retrying caller such as
@@ -71,6 +74,7 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
   protected boolean renew = false;
   private Scan scan;
   private int caching = 1;
+  protected final ClusterConnection cConnection;
   protected ScanMetrics scanMetrics;
   private boolean logScannerActivity = false;
   private int logCutOffLatency = 1000;
@@ -121,8 +125,9 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
    */
   public ScannerCallable(ClusterConnection connection, TableName tableName, Scan scan,
       ScanMetrics scanMetrics, RpcControllerFactory rpcControllerFactory, int id) {
-    super(connection, rpcControllerFactory, tableName, scan.getStartRow());
+    super(connection, tableName, scan.getStartRow());
     this.id = id;
+    this.cConnection = connection;
     this.scan = scan;
     this.scanMetrics = scanMetrics;
     Configuration conf = connection.getConfiguration();
@@ -180,16 +185,25 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
     }
   }
 
-  protected Result [] call(PayloadCarryingRpcController controller) throws Exception {
+
+  @Override
+  public Result [] call(int callTimeout) throws IOException {
     if (Thread.interrupted()) {
       throw new InterruptedIOException();
     }
-    if (this.closed) {
-      if (this.scannerId != -1) {
+
+    if (controller == null) {
+      controller = controllerFactory.newController();
+      controller.setPriority(getTableName());
+      controller.setCallTimeout(callTimeout);
+    }
+
+    if (closed) {
+      if (scannerId != -1) {
         close();
       }
     } else {
-      if (this.scannerId == -1L) {
+      if (scannerId == -1L) {
         this.scannerId = openScanner();
       } else {
         Result [] rrs = null;
@@ -198,56 +212,61 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
         setHeartbeatMessage(false);
         try {
           incRPCcallsMetrics();
-          request = RequestConverter.buildScanRequest(scannerId, caching, false, nextCallSeq,
+          request =
+              RequestConverter.buildScanRequest(scannerId, caching, false, nextCallSeq,
                 this.scanMetrics != null, renew);
           ScanResponse response = null;
-          response = getStub().scan(controller, request);
-          // Client and RS maintain a nextCallSeq number during the scan. Every next() call
-          // from client to server will increment this number in both sides. Client passes this
-          // number along with the request and at RS side both the incoming nextCallSeq and its
-          // nextCallSeq will be matched. In case of a timeout this increment at the client side
-          // should not happen. If at the server side fetching of next batch of data was over,
-          // there will be mismatch in the nextCallSeq number. Server will throw
-          // OutOfOrderScannerNextException and then client will reopen the scanner with startrow
-          // as the last successfully retrieved row.
-          // See HBASE-5974
-          nextCallSeq++;
-          long timestamp = System.currentTimeMillis();
-          setHeartbeatMessage(response.hasHeartbeatMessage() && response.getHeartbeatMessage());
-          // Results are returned via controller
-          CellScanner cellScanner = controller.cellScanner();
-          rrs = ResponseConverter.getResults(cellScanner, response);
-          if (logScannerActivity) {
-            long now = System.currentTimeMillis();
-            if (now - timestamp > logCutOffLatency) {
-              int rows = rrs == null ? 0 : rrs.length;
-              LOG.info("Took " + (now-timestamp) + "ms to fetch "
+          try {
+            response = getStub().scan(controller, request);
+            // Client and RS maintain a nextCallSeq number during the scan. Every next() call
+            // from client to server will increment this number in both sides. Client passes this
+            // number along with the request and at RS side both the incoming nextCallSeq and its
+            // nextCallSeq will be matched. In case of a timeout this increment at the client side
+            // should not happen. If at the server side fetching of next batch of data was over,
+            // there will be mismatch in the nextCallSeq number. Server will throw
+            // OutOfOrderScannerNextException and then client will reopen the scanner with startrow
+            // as the last successfully retrieved row.
+            // See HBASE-5974
+            nextCallSeq++;
+            long timestamp = System.currentTimeMillis();
+            setHeartbeatMessage(response.hasHeartbeatMessage() && response.getHeartbeatMessage());
+            // Results are returned via controller
+            CellScanner cellScanner = controller.cellScanner();
+            rrs = ResponseConverter.getResults(cellScanner, response);
+            if (logScannerActivity) {
+              long now = System.currentTimeMillis();
+              if (now - timestamp > logCutOffLatency) {
+                int rows = rrs == null ? 0 : rrs.length;
+                LOG.info("Took " + (now-timestamp) + "ms to fetch "
                   + rows + " rows from scanner=" + scannerId);
+              }
             }
-          }
-          updateServerSideMetrics(response);
-          // moreResults is only used for the case where a filter exhausts all elements
-          if (response.hasMoreResults() && !response.getMoreResults()) {
-            this.scannerId = -1L;
-            this.closed = true;
-            // Implied that no results were returned back, either.
-            return null;
-          }
-          // moreResultsInRegion explicitly defines when a RS may choose to terminate a batch due
-          // to size or quantity of results in the response.
-          if (response.hasMoreResultsInRegion()) {
-            // Set what the RS said
-            setHasMoreResultsContext(true);
-            setServerHasMoreResults(response.getMoreResultsInRegion());
-          } else {
-            // Server didn't respond whether it has more results or not.
-            setHasMoreResultsContext(false);
+            updateServerSideMetrics(response);
+            // moreResults is only used for the case where a filter exhausts all elements
+            if (response.hasMoreResults() && !response.getMoreResults()) {
+              scannerId = -1L;
+              closed = true;
+              // Implied that no results were returned back, either.
+              return null;
+            }
+            // moreResultsInRegion explicitly defines when a RS may choose to terminate a batch due
+            // to size or quantity of results in the response.
+            if (response.hasMoreResultsInRegion()) {
+              // Set what the RS said
+              setHasMoreResultsContext(true);
+              setServerHasMoreResults(response.getMoreResultsInRegion());
+            } else {
+              // Server didn't respond whether it has more results or not.
+              setHasMoreResultsContext(false);
+            }
+          } catch (ServiceException se) {
+            throw ProtobufUtil.getRemoteException(se);
           }
           updateResultsMetrics(rrs);
         } catch (IOException e) {
           if (logScannerActivity) {
-            LOG.info("Got exception making request " + ProtobufUtil.toText(request) + " to " +
-                getLocation(), e);
+            LOG.info("Got exception making request " + TextFormat.shortDebugString(request)
+              + " to " + getLocation(), e);
           }
           IOException ioe = e;
           if (e instanceof RemoteException) {
@@ -256,9 +275,9 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
           if (logScannerActivity && (ioe instanceof UnknownScannerException)) {
             try {
               HRegionLocation location =
-                  getConnection().relocateRegion(getTableName(), scan.getStartRow());
-              LOG.info("Scanner=" + scannerId + " expired, current region location is " +
-                  location.toString());
+                getConnection().relocateRegion(getTableName(), scan.getStartRow());
+              LOG.info("Scanner=" + scannerId
+                + " expired, current region location is " + location.toString());
             } catch (Throwable t) {
               LOG.info("Failed to relocate region", t);
             }
@@ -357,8 +376,8 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
           RequestConverter.buildScanRequest(this.scannerId, 0, true, this.scanMetrics != null);
       try {
         getStub().scan(controller, request);
-      } catch (Exception e) {
-        throw ProtobufUtil.handleRemoteException(e);
+      } catch (ServiceException se) {
+        throw ProtobufUtil.getRemoteException(se);
       }
     } catch (IOException e) {
       LOG.warn("Ignore, probably already closed", e);
@@ -368,8 +387,10 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
 
   protected long openScanner() throws IOException {
     incRPCcallsMetrics();
-    ScanRequest request = RequestConverter.buildScanRequest(
-        getLocation().getRegionInfo().getRegionName(), this.scan, 0, false);
+    ScanRequest request =
+      RequestConverter.buildScanRequest(
+        getLocation().getRegionInfo().getRegionName(),
+        this.scan, 0, false);
     try {
       ScanResponse response = getStub().scan(controller, request);
       long id = response.getScannerId();
@@ -378,8 +399,8 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
           + " on region " + getLocation().toString());
       }
       return id;
-    } catch (Exception e) {
-      throw ProtobufUtil.handleRemoteException(e);
+    } catch (ServiceException se) {
+      throw ProtobufUtil.getRemoteException(se);
     }
   }
 
@@ -422,6 +443,11 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
     return caching;
   }
 
+  @Override
+  public ClusterConnection getConnection() {
+    return cConnection;
+  }
+
   /**
    * Set the number of rows that will be fetched on next
    * @param caching the number of rows for caching
@@ -462,4 +488,4 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
   protected void setHasMoreResultsContext(boolean serverHasMoreResultsContext) {
     this.serverHasMoreResultsContext = serverHasMoreResultsContext;
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0206dc67/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SecureBulkLoadClient.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SecureBulkLoadClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SecureBulkLoadClient.java
index d6896e1..7b1547d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SecureBulkLoadClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SecureBulkLoadClient.java
@@ -22,9 +22,6 @@ import java.io.IOException;
 import java.util.List;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
-import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
@@ -41,35 +38,41 @@ import org.apache.hadoop.hbase.security.SecureBulkLoadUtil;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.security.token.Token;
 
+import com.google.protobuf.ServiceException;
+
 /**
  * Client proxy for SecureBulkLoadProtocol
  */
 @InterfaceAudience.Private
 public class SecureBulkLoadClient {
   private Table table;
-  private final RpcControllerFactory rpcControllerFactory;
 
-  public SecureBulkLoadClient(final Configuration conf, Table table) {
+  public SecureBulkLoadClient(Table table) {
     this.table = table;
-    this.rpcControllerFactory = new RpcControllerFactory(conf);
   }
 
   public String prepareBulkLoad(final Connection conn) throws IOException {
     try {
-      RegionServerCallable<String> callable = new RegionServerCallable<String>(conn,
-          this.rpcControllerFactory, table.getName(), HConstants.EMPTY_START_ROW) {
-        @Override
-        protected String call(PayloadCarryingRpcController controller) throws Exception {
-          byte[] regionName = getLocation().getRegionInfo().getRegionName();
-          RegionSpecifier region =
-              RequestConverter.buildRegionSpecifier(RegionSpecifierType.REGION_NAME, regionName);
-          PrepareBulkLoadRequest request = PrepareBulkLoadRequest.newBuilder()
-              .setTableName(ProtobufUtil.toProtoTableName(table.getName()))
-              .setRegion(region).build();
-          PrepareBulkLoadResponse response = getStub().prepareBulkLoad(null, request);
-          return response.getBulkToken();
-        }
-      };
+      RegionServerCallable<String> callable =
+          new RegionServerCallable<String>(conn, table.getName(), HConstants.EMPTY_START_ROW) {
+            @Override
+            public String call(int callTimeout) throws IOException {
+              byte[] regionName = getLocation().getRegionInfo().getRegionName();
+              RegionSpecifier region =
+                  RequestConverter
+                      .buildRegionSpecifier(RegionSpecifierType.REGION_NAME, regionName);
+              try {
+                PrepareBulkLoadRequest request =
+                    PrepareBulkLoadRequest.newBuilder()
+                        .setTableName(ProtobufUtil.toProtoTableName(table.getName()))
+                        .setRegion(region).build();
+                PrepareBulkLoadResponse response = getStub().prepareBulkLoad(null, request);
+                return response.getBulkToken();
+              } catch (ServiceException se) {
+                throw ProtobufUtil.getRemoteException(se);
+              }
+            }
+          };
       return RpcRetryingCallerFactory.instantiate(conn.getConfiguration(), null)
           .<String> newCaller().callWithRetries(callable, Integer.MAX_VALUE);
     } catch (Throwable throwable) {
@@ -79,19 +82,24 @@ public class SecureBulkLoadClient {
 
   public void cleanupBulkLoad(final Connection conn, final String bulkToken) throws IOException {
     try {
-      RegionServerCallable<Void> callable = new RegionServerCallable<Void>(conn,
-          this.rpcControllerFactory, table.getName(), HConstants.EMPTY_START_ROW) {
-        @Override
-        protected Void call(PayloadCarryingRpcController controller) throws Exception {
-          byte[] regionName = getLocation().getRegionInfo().getRegionName();
-          RegionSpecifier region = RequestConverter.buildRegionSpecifier(
-              RegionSpecifierType.REGION_NAME, regionName);
-          CleanupBulkLoadRequest request =
-              CleanupBulkLoadRequest.newBuilder().setRegion(region).setBulkToken(bulkToken).build();
-          getStub().cleanupBulkLoad(null, request);
-          return null;
-        }
-      };
+      RegionServerCallable<Void> callable =
+          new RegionServerCallable<Void>(conn, table.getName(), HConstants.EMPTY_START_ROW) {
+            @Override
+            public Void call(int callTimeout) throws IOException {
+              byte[] regionName = getLocation().getRegionInfo().getRegionName();
+              RegionSpecifier region = RequestConverter.buildRegionSpecifier(
+                RegionSpecifierType.REGION_NAME, regionName);
+              try {
+                CleanupBulkLoadRequest request =
+                    CleanupBulkLoadRequest.newBuilder().setRegion(region)
+                        .setBulkToken(bulkToken).build();
+                getStub().cleanupBulkLoad(null, request);
+              } catch (ServiceException se) {
+                throw ProtobufUtil.getRemoteException(se);
+              }
+              return null;
+            }
+          };
       RpcRetryingCallerFactory.instantiate(conn.getConfiguration(), null)
           .<Void> newCaller().callWithRetries(callable, Integer.MAX_VALUE);
     } catch (Throwable throwable) {
@@ -122,12 +130,12 @@ public class SecureBulkLoadClient {
     try {
       BulkLoadHFileResponse response = client.bulkLoadHFile(null, request);
       return response.getLoaded();
-    } catch (Exception se) {
-      throw ProtobufUtil.handleRemoteException(se);
+    } catch (ServiceException se) {
+      throw ProtobufUtil.getRemoteException(se);
     }
   }
 
   public Path getStagingPath(String bulkToken, byte[] family) throws IOException {
     return SecureBulkLoadUtil.getStagingPath(table.getConfiguration(), bulkToken, family);
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0206dc67/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/MasterCoprocessorRpcChannel.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/MasterCoprocessorRpcChannel.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/MasterCoprocessorRpcChannel.java
index a6384e3..6fae5cb 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/MasterCoprocessorRpcChannel.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/MasterCoprocessorRpcChannel.java
@@ -77,4 +77,5 @@ public class MasterCoprocessorRpcChannel extends SyncCoprocessorRpcChannel {
     }
     return response;
   }
-}
\ No newline at end of file
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0206dc67/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/PayloadCarryingRpcController.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/PayloadCarryingRpcController.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/PayloadCarryingRpcController.java
index 6c290a6..f4f18b3 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/PayloadCarryingRpcController.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/PayloadCarryingRpcController.java
@@ -17,39 +17,24 @@
  */
 package org.apache.hadoop.hbase.ipc;
 
-import java.io.IOException;
 import java.util.List;
-import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.hadoop.hbase.CellScannable;
 import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.util.Bytes;
-
-import com.google.protobuf.RpcCallback;
-import com.google.protobuf.RpcController;
 
 /**
  * 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.
+ * 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.  This class is used ferrying data across the proxy/protobuf
+ * service chasm.  Used by client and server ipc'ing.
  */
 @InterfaceAudience.Private
-public class PayloadCarryingRpcController implements RpcController, CellScannable {
-  /**
-   * The time, in ms before the call should expire.
-   */
-  protected volatile Integer callTimeout;
-  protected volatile boolean cancelled = false;
-  protected final AtomicReference<RpcCallback<Object>> cancellationCb = new AtomicReference<>(null);
-  protected final AtomicReference<RpcCallback<IOException>> failureCb = new AtomicReference<>(null);
-  private IOException exception;
+public class PayloadCarryingRpcController
+    extends TimeLimitedRpcController implements CellScannable {
 
   public static final int PRIORITY_UNSET = -1;
   /**
@@ -108,123 +93,15 @@ public class PayloadCarryingRpcController implements RpcController, CellScannabl
   }
 
   /**
-   * @param regionName RegionName. If hbase:meta, we'll set high priority.
-   */
-  public void setPriority(final byte [] regionName) {
-    if (isMetaRegion(regionName)) {
-      setPriority(TableName.META_TABLE_NAME);
-    }
-  }
-
-  private static boolean isMetaRegion(final byte[] regionName) {
-    return Bytes.equals(regionName, HRegionInfo.FIRST_META_REGIONINFO.getRegionName())
-        || Bytes.equals(regionName, HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes());
-  }
-
-  /**
    * @return The priority of this request
    */
   public int getPriority() {
     return priority;
   }
 
-  @Override
-  public void reset() {
+  @Override public void reset() {
+    super.reset();
     priority = 0;
     cellScanner = null;
-    exception = null;
-    cancelled = false;
-    failureCb.set(null);
-    cancellationCb.set(null);
-    callTimeout = null;
-  }
-
-  public int getCallTimeout() {
-    if (callTimeout != null) {
-      return callTimeout;
-    } else {
-      return 0;
-    }
-  }
-
-  public void setCallTimeout(int callTimeout) {
-    this.callTimeout = callTimeout;
-  }
-
-  public boolean hasCallTimeout(){
-    return callTimeout != null;
-  }
-
-  @Override
-  public String errorText() {
-    if (exception != null) {
-      return exception.getMessage();
-    } else {
-      return null;
-    }
-  }
-
-  /**
-   * For use in async rpc clients
-   * @return true if failed
-   */
-  @Override
-  public boolean failed() {
-    return this.exception != null;
-  }
-
-  @Override
-  public boolean isCanceled() {
-    return cancelled;
-  }
-
-  @Override
-  public void notifyOnCancel(RpcCallback<Object> cancellationCb) {
-    this.cancellationCb.set(cancellationCb);
-    if (this.cancelled) {
-      cancellationCb.run(null);
-    }
-  }
-
-  /**
-   * Notify a callback on error.
-   * For use in async rpc clients
-   *
-   * @param failureCb the callback to call on error
-   */
-  public void notifyOnFail(RpcCallback<IOException> failureCb) {
-    this.failureCb.set(failureCb);
-    if (this.exception != null) {
-      failureCb.run(this.exception);
-    }
-  }
-
-  @Override
-  public void setFailed(String reason) {
-    this.exception = new IOException(reason);
-    if (this.failureCb.get() != null) {
-      this.failureCb.get().run(this.exception);
-    }
-  }
-
-  /**
-   * Set failed with an exception to pass on.
-   * For use in async rpc clients
-   *
-   * @param e exception to set with
-   */
-  public void setFailed(IOException e) {
-    this.exception = e;
-    if (this.failureCb.get() != null) {
-      this.failureCb.get().run(this.exception);
-    }
-  }
-
-  @Override
-  public void startCancel() {
-    cancelled = true;
-    if (cancellationCb.get() != null) {
-      cancellationCb.get().run(null);
-    }
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0206dc67/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionCoprocessorRpcChannel.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionCoprocessorRpcChannel.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionCoprocessorRpcChannel.java
index dbc9041..55d6375 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionCoprocessorRpcChannel.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionCoprocessorRpcChannel.java
@@ -76,23 +76,30 @@ public class RegionCoprocessorRpcChannel extends SyncCoprocessorRpcChannel {
       Descriptors.MethodDescriptor method, Message request, Message responsePrototype)
           throws IOException {
     if (LOG.isTraceEnabled()) {
-      LOG.trace("Call: " + method.getName() + ", " + request.toString());
+      LOG.trace("Call: "+method.getName()+", "+request.toString());
     }
+
     if (row == null) {
       throw new IllegalArgumentException("Missing row property for remote region location");
     }
+
+    final RpcController rpcController = controller == null
+        ? rpcControllerFactory.newController() : controller;
+
     final ClientProtos.CoprocessorServiceCall call =
         CoprocessorRpcUtils.buildServiceCall(row, method, request);
     RegionServerCallable<CoprocessorServiceResponse> callable =
-        new RegionServerCallable<CoprocessorServiceResponse>(connection,
-          controller == null? this.rpcControllerFactory.newController():
-            (PayloadCarryingRpcController)controller,
-          table, row) {
+        new RegionServerCallable<CoprocessorServiceResponse>(connection, table, row) {
       @Override
-      protected CoprocessorServiceResponse call(PayloadCarryingRpcController controller)
-      throws Exception {
+      public CoprocessorServiceResponse call(int callTimeout) throws Exception {
+        if (rpcController instanceof PayloadCarryingRpcController) {
+          ((PayloadCarryingRpcController) rpcController).setPriority(tableName);
+        }
+        if (rpcController instanceof TimeLimitedRpcController) {
+          ((TimeLimitedRpcController) rpcController).setCallTimeout(callTimeout);
+        }
         byte[] regionName = getLocation().getRegionInfo().getRegionName();
-        return ProtobufUtil.execService(getRpcController(), getStub(), call, regionName);
+        return ProtobufUtil.execService(rpcController, getStub(), call, regionName);
       }
     };
     CoprocessorServiceResponse result = rpcCallerFactory.<CoprocessorServiceResponse> newCaller()


[14/50] [abbrv] hbase git commit: HBASE-16308 Contain protobuf references Gather up the pb references into a few locations only rather than have pb references distributed all about the code base.

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/ed87a81b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/TimeLimitedRpcController.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/TimeLimitedRpcController.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/TimeLimitedRpcController.java
deleted file mode 100644
index cf08ea9..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/TimeLimitedRpcController.java
+++ /dev/null
@@ -1,142 +0,0 @@
-/**
- * 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
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.ipc;
-
-import com.google.protobuf.RpcCallback;
-import com.google.protobuf.RpcController;
-
-import java.io.IOException;
-import java.util.concurrent.atomic.AtomicReference;
-
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-
-@InterfaceAudience.Private
-public class TimeLimitedRpcController implements RpcController {
-
-  /**
-   * The time, in ms before the call should expire.
-   */
-  protected volatile Integer callTimeout;
-  protected volatile boolean cancelled = false;
-  protected final AtomicReference<RpcCallback<Object>> cancellationCb =
-      new AtomicReference<>(null);
-
-  protected final AtomicReference<RpcCallback<IOException>> failureCb =
-      new AtomicReference<>(null);
-
-  private IOException exception;
-
-  public int getCallTimeout() {
-    if (callTimeout != null) {
-      return callTimeout;
-    } else {
-      return 0;
-    }
-  }
-
-  public void setCallTimeout(int callTimeout) {
-    this.callTimeout = callTimeout;
-  }
-
-  public boolean hasCallTimeout(){
-    return callTimeout != null;
-  }
-
-  @Override
-  public String errorText() {
-    if (exception != null) {
-      return exception.getMessage();
-    } else {
-      return null;
-    }
-  }
-
-  /**
-   * For use in async rpc clients
-   * @return true if failed
-   */
-  @Override
-  public boolean failed() {
-    return this.exception != null;
-  }
-
-  @Override
-  public boolean isCanceled() {
-    return cancelled;
-  }
-
-  @Override
-  public void notifyOnCancel(RpcCallback<Object> cancellationCb) {
-    this.cancellationCb.set(cancellationCb);
-    if (this.cancelled) {
-      cancellationCb.run(null);
-    }
-  }
-
-  /**
-   * Notify a callback on error.
-   * For use in async rpc clients
-   *
-   * @param failureCb the callback to call on error
-   */
-  public void notifyOnFail(RpcCallback<IOException> failureCb) {
-    this.failureCb.set(failureCb);
-    if (this.exception != null) {
-      failureCb.run(this.exception);
-    }
-  }
-
-  @Override
-  public void reset() {
-    exception = null;
-    cancelled = false;
-    failureCb.set(null);
-    cancellationCb.set(null);
-    callTimeout = null;
-  }
-
-  @Override
-  public void setFailed(String reason) {
-    this.exception = new IOException(reason);
-    if (this.failureCb.get() != null) {
-      this.failureCb.get().run(this.exception);
-    }
-  }
-
-  /**
-   * Set failed with an exception to pass on.
-   * For use in async rpc clients
-   *
-   * @param e exception to set with
-   */
-  public void setFailed(IOException e) {
-    this.exception = e;
-    if (this.failureCb.get() != null) {
-      this.failureCb.get().run(this.exception);
-    }
-  }
-
-  @Override
-  public void startCancel() {
-    cancelled = true;
-    if (cancellationCb.get() != null) {
-      cancellationCb.get().run(null);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ed87a81b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
index 5ba0572..623acd5 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
@@ -17,6 +17,9 @@
  */
 package org.apache.hadoop.hbase.protobuf;
 
+import static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier
+.RegionSpecifierType.REGION_NAME;
+
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
@@ -38,14 +41,11 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.NavigableSet;
 import java.util.Set;
+import java.util.concurrent.Callable;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
-
-import static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier
-.RegionSpecifierType.REGION_NAME;
-
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.CellUtil;
@@ -53,6 +53,7 @@ import org.apache.hadoop.hbase.ClusterId;
 import org.apache.hadoop.hbase.ClusterStatus;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HBaseIOException;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
@@ -124,8 +125,8 @@ import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos;
 import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.LiveServerInfo;
 import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionInTransition;
 import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionLoad;
-import org.apache.hadoop.hbase.protobuf.generated.FSProtos.HBaseVersionFileContent;
 import org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos;
+import org.apache.hadoop.hbase.protobuf.generated.FSProtos.HBaseVersionFileContent;
 import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair;
@@ -171,11 +172,9 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.DynamicClassLoader;
 import org.apache.hadoop.hbase.util.ExceptionUtil;
 import org.apache.hadoop.hbase.util.Methods;
-import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.VersionInfo;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 
 import com.google.common.collect.ArrayListMultimap;
@@ -334,17 +333,32 @@ public final class ProtobufUtil {
    *   a new IOException that wraps the unexpected ServiceException.
    */
   public static IOException getRemoteException(ServiceException se) {
-    Throwable e = se.getCause();
-    if (e == null) {
-      return new IOException(se);
+    return makeIOExceptionOfException(se);
+  }
+
+  /**
+   * Like {@link #getRemoteException(ServiceException)} but more generic, able to handle more than
+   * just {@link ServiceException}. Prefer this method to
+   * {@link #getRemoteException(ServiceException)} because trying to
+   * contain direct protobuf references.
+   * @param e
+   */
+  public static IOException handleRemoteException(Exception e) {
+    return makeIOExceptionOfException(e);
+  }
+
+  private static IOException makeIOExceptionOfException(Exception e) {
+    Throwable t = e;
+    if (e instanceof ServiceException) {
+      t = e.getCause();
     }
-    if (ExceptionUtil.isInterrupt(e)) {
-      return ExceptionUtil.asInterrupt(e);
+    if (ExceptionUtil.isInterrupt(t)) {
+      return ExceptionUtil.asInterrupt(t);
     }
-    if (e instanceof RemoteException) {
-      e = ((RemoteException) e).unwrapRemoteException();
+    if (t instanceof RemoteException) {
+      t = ((RemoteException)t).unwrapRemoteException();
     }
-    return e instanceof IOException ? (IOException) e : new IOException(se);
+    return t instanceof IOException? (IOException)t: new HBaseIOException(t);
   }
 
   /**
@@ -1252,7 +1266,6 @@ public final class ProtobufUtil {
     return toMutation(type, mutation, builder, HConstants.NO_NONCE);
   }
 
-  @SuppressWarnings("deprecation")
   public static MutationProto toMutation(final MutationType type, final Mutation mutation,
       MutationProto.Builder builder, long nonce)
   throws IOException {
@@ -2658,13 +2671,11 @@ public final class ProtobufUtil {
     }
   }
 
-  @SuppressWarnings("deprecation")
   public static CompactionDescriptor toCompactionDescriptor(HRegionInfo info, byte[] family,
       List<Path> inputPaths, List<Path> outputPaths, Path storeDir) {
     return toCompactionDescriptor(info, null, family, inputPaths, outputPaths, storeDir);
   }
 
-  @SuppressWarnings("deprecation")
   public static CompactionDescriptor toCompactionDescriptor(HRegionInfo info, byte[] regionName,
       byte[] family, List<Path> inputPaths, List<Path> outputPaths, Path storeDir) {
     // compaction descriptor contains relative paths.
@@ -3663,4 +3674,28 @@ public final class ProtobufUtil {
     return new RegionLoadStats(stats.getMemstoreLoad(), stats.getHeapOccupancy(),
         stats.getCompactionPressure());
   }
-}
+
+  /**
+   * @param msg
+   * @return A String version of the passed in <code>msg</code>
+   */
+  public static String toText(Message msg) {
+    return TextFormat.shortDebugString(msg);
+  }
+
+  public static byte [] toBytes(ByteString bs) {
+    return bs.toByteArray();
+  }
+
+  /**
+   * Contain ServiceException inside here. Take a callable that is doing our pb rpc and run it.
+   * @throws IOException
+   */
+  public static <T> T call(Callable<T> callable) throws IOException {
+    try {
+      return callable.call();
+    } catch (Exception e) {
+      throw ProtobufUtil.handleRemoteException(e);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/ed87a81b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientScanner.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientScanner.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientScanner.java
index f083001..fd2a393 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientScanner.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientScanner.java
@@ -65,7 +65,6 @@ public class TestClientScanner {
   RpcControllerFactory controllerFactory;
 
   @Before
-  @SuppressWarnings("deprecation")
   public void setup() throws IOException {
     clusterConn = Mockito.mock(ClusterConnection.class);
     rpcFactory = Mockito.mock(RpcRetryingCallerFactory.class);

http://git-wip-us.apache.org/repos/asf/hbase/blob/ed87a81b/hbase-common/src/main/java/org/apache/hadoop/hbase/HBaseIOException.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HBaseIOException.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HBaseIOException.java
index 9c3367e..edcbdc5 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HBaseIOException.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HBaseIOException.java
@@ -45,4 +45,5 @@ public class HBaseIOException extends IOException {
 
   public HBaseIOException(Throwable cause) {
       super(cause);
-  }}
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/ed87a81b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ExceptionUtil.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ExceptionUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ExceptionUtil.java
index 688b51a..7e6c5d6 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ExceptionUtil.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ExceptionUtil.java
@@ -73,4 +73,4 @@ public class ExceptionUtil {
 
   private ExceptionUtil() {
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/ed87a81b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
index 73226aa..ec28315 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
@@ -75,20 +75,17 @@ import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseIOException;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.Server;
-import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
-import org.apache.hadoop.hbase.client.Operation;
 import org.apache.hadoop.hbase.client.VersionInfoUtil;
 import org.apache.hadoop.hbase.codec.Codec;
 import org.apache.hadoop.hbase.conf.ConfigurationObserver;
 import org.apache.hadoop.hbase.exceptions.RegionMovedException;
 import org.apache.hadoop.hbase.io.ByteBufferInputStream;
+import org.apache.hadoop.hbase.io.ByteBufferListOutputStream;
 import org.apache.hadoop.hbase.io.ByteBufferOutputStream;
 import org.apache.hadoop.hbase.io.ByteBufferPool;
-import org.apache.hadoop.hbase.io.ByteBufferListOutputStream;
 import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler;
 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
@@ -99,7 +96,6 @@ import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse;
 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader;
 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader;
 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation;
-import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.security.AccessDeniedException;
 import org.apache.hadoop.hbase.security.AuthMethod;
 import org.apache.hadoop.hbase.security.HBasePolicyProvider;

http://git-wip-us.apache.org/repos/asf/hbase/blob/ed87a81b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
index 09dedec..8ddbe18 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
@@ -87,6 +87,8 @@ import org.apache.hadoop.hbase.io.hfile.HFileContext;
 import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
 import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder;
 import org.apache.hadoop.hbase.io.hfile.HFileScanner;
+import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
+import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.regionserver.BloomType;
 import org.apache.hadoop.hbase.regionserver.HStore;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
@@ -102,7 +104,6 @@ import org.apache.hadoop.util.ToolRunner;
 
 /**
  * Tool to load the output of HFileOutputFormat into an existing table.
- * @see #usage()
  */
 @InterfaceAudience.Public
 @InterfaceStability.Stable
@@ -130,11 +131,13 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
   private String bulkToken;
   private UserProvider userProvider;
   private int nrThreads;
+  private RpcControllerFactory rpcControllerFactory;
 
   private LoadIncrementalHFiles() {}
 
   public LoadIncrementalHFiles(Configuration conf) throws Exception {
     super(conf);
+    this.rpcControllerFactory = new RpcControllerFactory(conf);
     initialize();
   }
 
@@ -322,7 +325,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
     // LQI queue does not need to be threadsafe -- all operations on this queue
     // happen in this thread
     Deque<LoadQueueItem> queue = new LinkedList<>();
-    SecureBulkLoadClient secureClient =  new SecureBulkLoadClient(table);
+    SecureBulkLoadClient secureClient =  new SecureBulkLoadClient(table.getConfiguration(), table);
 
     try {
       /*
@@ -473,9 +476,11 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
 
   /**
    * Used by the replication sink to load the hfiles from the source cluster. It does the following,
-   * 1. {@link LoadIncrementalHFiles#groupOrSplitPhase(Table, ExecutorService, Deque, Pair)} 2.
-   * {@link
-   * LoadIncrementalHFiles#bulkLoadPhase(Table, Connection, ExecutorService, Deque, Multimap)}
+   * <ol>
+   * <li>LoadIncrementalHFiles#groupOrSplitPhase(Table, ExecutorService, Deque, Pair)}</li>
+   * <li>LoadIncrementalHFiles#bulkLoadPhase(Table, Connection, ExecutorService, Deque, Multimap)
+   * </li>
+   * </ol>
    * @param table Table to which these hfiles should be loaded to
    * @param conn Connection to use
    * @param queue {@link LoadQueueItem} has hfiles yet to be loaded
@@ -776,27 +781,23 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
   protected List<LoadQueueItem> tryAtomicRegionLoad(final Connection conn,
       final TableName tableName, final byte[] first, final Collection<LoadQueueItem> lqis)
   throws IOException {
-    final List<Pair<byte[], String>> famPaths =
-      new ArrayList<>(lqis.size());
+    final List<Pair<byte[], String>> famPaths = new ArrayList<>(lqis.size());
     for (LoadQueueItem lqi : lqis) {
       famPaths.add(Pair.newPair(lqi.family, lqi.hfilePath.toString()));
     }
-
-    final RegionServerCallable<Boolean> svrCallable =
-        new RegionServerCallable<Boolean>(conn, tableName, first) {
+    final RegionServerCallable<Boolean> svrCallable = new RegionServerCallable<Boolean>(conn,
+        rpcControllerFactory, tableName, first) {
       @Override
-      public Boolean call(int callTimeout) throws Exception {
+      protected Boolean call(PayloadCarryingRpcController controller) throws Exception {
         SecureBulkLoadClient secureClient = null;
         boolean success = false;
-
         try {
           LOG.debug("Going to connect to server " + getLocation() + " for row "
               + Bytes.toStringBinary(getRow()) + " with hfile group " + famPaths);
           byte[] regionName = getLocation().getRegionInfo().getRegionName();
           try (Table table = conn.getTable(getTableName())) {
-            secureClient = new SecureBulkLoadClient(table);
-            success =
-                secureClient.secureBulkLoadHFiles(getStub(), famPaths, regionName,
+            secureClient = new SecureBulkLoadClient(getConf(), table);
+            success = secureClient.secureBulkLoadHFiles(getStub(), famPaths, regionName,
                   assignSeqIds, fsDelegationToken.getUserToken(), bulkToken);
           }
           return success;
@@ -1078,7 +1079,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
 
   /**
    * Called from replication sink, where it manages bulkToken(staging directory) by itself. This is
-   * used only when {@link SecureBulkLoadEndpoint} is configured in hbase.coprocessor.region.classes
+   * used only when SecureBulkLoadEndpoint is configured in hbase.coprocessor.region.classes
    * property. This directory is used as a temporary directory where all files are initially
    * copied/moved from user given directory, set all the required file permissions and then from
    * their it is finally loaded into a table. This should be set only when, one would like to manage
@@ -1088,5 +1089,4 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
   public void setBulkToken(String stagingDir) {
     this.bulkToken = stagingDir;
   }
-
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/ed87a81b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ExpiredMobFileCleanerChore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ExpiredMobFileCleanerChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ExpiredMobFileCleanerChore.java
index a21edcc..3261bd6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ExpiredMobFileCleanerChore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ExpiredMobFileCleanerChore.java
@@ -35,8 +35,6 @@ import org.apache.hadoop.hbase.mob.ExpiredMobFileCleaner;
 import org.apache.hadoop.hbase.mob.MobConstants;
 import org.apache.hadoop.hbase.mob.MobUtils;
 
-import com.google.protobuf.ServiceException;
-
 /**
  * The Class ExpiredMobFileCleanerChore for running cleaner regularly to remove the expired
  * mob files.
@@ -86,10 +84,6 @@ public class ExpiredMobFileCleanerChore extends ScheduledChore {
             } catch (LockTimeoutException e) {
               LOG.info("Fail to acquire the lock because of timeout, maybe a"
                 + " MobCompactor is running", e);
-            } catch (ServiceException e) {
-              LOG.error(
-                "Fail to clean the expired mob files for the column " + hcd.getNameAsString()
-                  + " in the table " + htd.getNameAsString(), e);
             } catch (IOException e) {
               LOG.error(
                 "Fail to clean the expired mob files for the column " + hcd.getNameAsString()

http://git-wip-us.apache.org/repos/asf/hbase/blob/ed87a81b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
index 531883a..d7ba4f3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
@@ -61,9 +61,9 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionRequest;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionResponse;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ServerInfo;
-import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
 import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds;
 import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.StoreSequenceId;
+import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
@@ -454,8 +454,7 @@ public class ServerManager {
   /**
    * Adds the onlineServers list. onlineServers should be locked.
    * @param serverName The remote servers name.
-   * @param sl
-   * @return Server load from the removed server, if any.
+   * @param s
    */
   @VisibleForTesting
   void recordNewServerWithLock(final ServerName serverName, final ServerLoad sl) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/ed87a81b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/ExpiredMobFileCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/ExpiredMobFileCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/ExpiredMobFileCleaner.java
index 3c965cb..d4a54bb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/ExpiredMobFileCleaner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/ExpiredMobFileCleaner.java
@@ -41,8 +41,6 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 
-import com.google.protobuf.ServiceException;
-
 /**
  * The cleaner to delete the expired MOB files.
  */
@@ -60,11 +58,8 @@ public class ExpiredMobFileCleaner extends Configured implements Tool {
    * directory.
    * @param tableName The current table name.
    * @param family The current family.
-   * @throws ServiceException
-   * @throws IOException
    */
-  public void cleanExpiredMobFiles(String tableName, HColumnDescriptor family)
-      throws ServiceException, IOException {
+  public void cleanExpiredMobFiles(String tableName, HColumnDescriptor family) throws IOException {
     Configuration conf = getConf();
     TableName tn = TableName.valueOf(tableName);
     FileSystem fs = FileSystem.get(conf);
@@ -99,7 +94,7 @@ public class ExpiredMobFileCleaner extends Configured implements Tool {
     String tableName = args[0];
     String familyName = args[1];
     TableName tn = TableName.valueOf(tableName);
-    HBaseAdmin.checkHBaseAvailable(getConf());
+    HBaseAdmin.available(getConf());
     Connection connection = ConnectionFactory.createConnection(getConf());
     Admin admin = connection.getAdmin();
     try {
@@ -127,5 +122,4 @@ public class ExpiredMobFileCleaner extends Configured implements Tool {
       }
     }
   }
-
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/ed87a81b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/mapreduce/Sweeper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/mapreduce/Sweeper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/mapreduce/Sweeper.java
index 8547c8c..c27e8ae 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/mapreduce/Sweeper.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/mapreduce/Sweeper.java
@@ -38,8 +38,6 @@ import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.zookeeper.KeeperException;
 
-import com.google.protobuf.ServiceException;
-
 /**
  * The sweep tool. It deletes the mob files that are not used and merges the small mob files to
  * bigger ones. Each run of this sweep tool only handles one column family. The runs on
@@ -64,10 +62,10 @@ public class Sweeper extends Configured implements Tool {
    * @throws ServiceException
    */
   int sweepFamily(String tableName, String familyName) throws IOException, InterruptedException,
-      ClassNotFoundException, KeeperException, ServiceException {
+      ClassNotFoundException, KeeperException {
     Configuration conf = getConf();
     // make sure the target HBase exists.
-    HBaseAdmin.checkHBaseAvailable(conf);
+    HBaseAdmin.available(conf);
     Connection connection = ConnectionFactory.createConnection(getConf());
     Admin admin = connection.getAdmin();
     try {

http://git-wip-us.apache.org/repos/asf/hbase/blob/ed87a81b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index d87ada4..fb9a605 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -87,7 +87,6 @@ import org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface;
 import org.apache.hadoop.hbase.ipc.RpcServerInterface;
 import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
 import org.apache.hadoop.hbase.ipc.ServerRpcController;
-import org.apache.hadoop.hbase.ipc.TimeLimitedRpcController;
 import org.apache.hadoop.hbase.master.MasterRpcServices;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.RequestConverter;
@@ -2765,13 +2764,15 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
                     timeLimitDelta =
                         scannerLeaseTimeoutPeriod > 0 ? scannerLeaseTimeoutPeriod : rpcTimeout;
                   }
-                  if (controller instanceof TimeLimitedRpcController) {
-                    TimeLimitedRpcController timeLimitedRpcController =
-                        (TimeLimitedRpcController)controller;
-                    if (timeLimitedRpcController.getCallTimeout() > 0) {
-                      timeLimitDelta = Math.min(timeLimitDelta,
-                          timeLimitedRpcController.getCallTimeout());
+                  if (controller instanceof PayloadCarryingRpcController) {
+                    PayloadCarryingRpcController pRpcController =
+                        (PayloadCarryingRpcController)controller;
+                    if (pRpcController.getCallTimeout() > 0) {
+                      timeLimitDelta = Math.min(timeLimitDelta, pRpcController.getCallTimeout());
                     }
+                  } else {
+                    throw new UnsupportedOperationException("We only do " +
+                      "PayloadCarryingRpcControllers! FIX IF A PROBLEM");
                   }
                   // Use half of whichever timeout value was more restrictive... But don't allow
                   // the time limit to be less than the allowable minimum (could cause an

http://git-wip-us.apache.org/repos/asf/hbase/blob/ed87a81b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEditsReplaySink.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEditsReplaySink.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEditsReplaySink.java
index 3eb85bd..c71153d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEditsReplaySink.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEditsReplaySink.java
@@ -18,8 +18,6 @@
  */
 package org.apache.hadoop.hbase.regionserver.wal;
 
-import com.google.protobuf.ServiceException;
-
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
@@ -44,7 +42,6 @@ import org.apache.hadoop.hbase.client.RegionServerCallable;
 import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
 import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
-import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.ReplicationProtbufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
@@ -61,10 +58,8 @@ import org.apache.hadoop.hbase.wal.WAL.Entry;
  */
 @InterfaceAudience.Private
 public class WALEditsReplaySink {
-
   private static final Log LOG = LogFactory.getLog(WALEditsReplaySink.class);
   private static final int MAX_BATCH_SIZE = 1024;
-
   private final Configuration conf;
   private final ClusterConnection conn;
   private final TableName tableName;
@@ -166,8 +161,8 @@ public class WALEditsReplaySink {
     try {
       RpcRetryingCallerFactory factory = RpcRetryingCallerFactory.instantiate(conf, null);
       ReplayServerCallable<ReplicateWALEntryResponse> callable =
-          new ReplayServerCallable<ReplicateWALEntryResponse>(this.conn, this.tableName, regionLoc,
-              regionInfo, entries);
+          new ReplayServerCallable<ReplicateWALEntryResponse>(this.conn, this.rpcControllerFactory,
+              this.tableName, regionLoc, entries);
       factory.<ReplicateWALEntryResponse> newCaller().callWithRetries(callable, this.replayTimeout);
     } catch (IOException ie) {
       if (skipErrors) {
@@ -184,31 +179,19 @@ public class WALEditsReplaySink {
    * @param <R>
    */
   class ReplayServerCallable<R> extends RegionServerCallable<ReplicateWALEntryResponse> {
-    private HRegionInfo regionInfo;
     private List<Entry> entries;
 
-    ReplayServerCallable(final Connection connection, final TableName tableName,
-        final HRegionLocation regionLoc, final HRegionInfo regionInfo,
-        final List<Entry> entries) {
-      super(connection, tableName, null);
+    ReplayServerCallable(final Connection connection, RpcControllerFactory rpcControllerFactory,
+        final TableName tableName, final HRegionLocation regionLoc, final List<Entry> entries) {
+      super(connection, rpcControllerFactory, tableName, null);
       this.entries = entries;
-      this.regionInfo = regionInfo;
       setLocation(regionLoc);
     }
 
     @Override
-    public ReplicateWALEntryResponse call(int callTimeout) throws IOException {
-      try {
-        replayToServer(this.regionInfo, this.entries);
-      } catch (ServiceException se) {
-        throw ProtobufUtil.getRemoteException(se);
-      }
-      return null;
-    }
-
-    private void replayToServer(HRegionInfo regionInfo, List<Entry> entries)
-        throws IOException, ServiceException {
-      if (entries.isEmpty()) return;
+    protected ReplicateWALEntryResponse call(PayloadCarryingRpcController controller)
+    throws Exception {
+      if (entries.isEmpty()) return null;
 
       Entry[] entriesArray = new Entry[entries.size()];
       entriesArray = entries.toArray(entriesArray);
@@ -216,12 +199,8 @@ public class WALEditsReplaySink {
 
       Pair<AdminProtos.ReplicateWALEntryRequest, CellScanner> p =
           ReplicationProtbufUtil.buildReplicateWALEntryRequest(entriesArray);
-      PayloadCarryingRpcController controller = rpcControllerFactory.newController(p.getSecond());
-      try {
-        remoteSvr.replay(controller, p.getFirst());
-      } catch (ServiceException se) {
-        throw ProtobufUtil.getRemoteException(se);
-      }
+      controller.setCellScanner(p.getSecond());
+      return remoteSvr.replay(controller, p.getFirst());
     }
 
     @Override
@@ -245,4 +224,4 @@ public class WALEditsReplaySink {
       }
     }
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/ed87a81b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java
index b0fd176..c756294 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java
@@ -34,7 +34,6 @@ import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.CellScanner;
@@ -46,27 +45,21 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.RegionLocations;
+import org.apache.hadoop.hbase.TableDescriptors;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.hadoop.hbase.TableDescriptors;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.RegionAdminServiceCallable;
-import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
 import org.apache.hadoop.hbase.client.RetryingCallable;
 import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
 import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
-import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.ReplicationProtbufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryResponse;
-import org.apache.hadoop.hbase.wal.WAL.Entry;
-import org.apache.hadoop.hbase.wal.WALSplitter.EntryBuffers;
-import org.apache.hadoop.hbase.wal.WALSplitter.OutputSink;
-import org.apache.hadoop.hbase.wal.WALSplitter.PipelineController;
-import org.apache.hadoop.hbase.wal.WALSplitter.RegionEntryBuffer;
-import org.apache.hadoop.hbase.wal.WALSplitter.SinkWriter;
 import org.apache.hadoop.hbase.replication.BaseWALEntryFilter;
 import org.apache.hadoop.hbase.replication.ChainWALEntryFilter;
 import org.apache.hadoop.hbase.replication.HBaseReplicationEndpoint;
@@ -74,12 +67,17 @@ import org.apache.hadoop.hbase.replication.WALEntryFilter;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.Threads;
+import org.apache.hadoop.hbase.wal.WAL.Entry;
+import org.apache.hadoop.hbase.wal.WALSplitter.EntryBuffers;
+import org.apache.hadoop.hbase.wal.WALSplitter.OutputSink;
+import org.apache.hadoop.hbase.wal.WALSplitter.PipelineController;
+import org.apache.hadoop.hbase.wal.WALSplitter.RegionEntryBuffer;
+import org.apache.hadoop.hbase.wal.WALSplitter.SinkWriter;
 import org.apache.hadoop.util.StringUtils;
 
 import com.google.common.cache.Cache;
 import com.google.common.cache.CacheBuilder;
 import com.google.common.collect.Lists;
-import com.google.protobuf.ServiceException;
 
 /**
  * A {@link org.apache.hadoop.hbase.replication.ReplicationEndpoint} endpoint
@@ -611,9 +609,8 @@ public class RegionReplicaReplicationEndpoint extends HBaseReplicationEndpoint {
    * Calls replay on the passed edits for the given set of entries belonging to the region. It skips
    * the entry if the region boundaries have changed or the region is gone.
    */
-  static class RegionReplicaReplayCallable
-    extends RegionAdminServiceCallable<ReplicateWALEntryResponse> {
-
+  static class RegionReplicaReplayCallable extends
+      RegionAdminServiceCallable<ReplicateWALEntryResponse> {
     private final List<Entry> entries;
     private final byte[] initialEncodedRegionName;
     private final AtomicLong skippedEntries;
@@ -628,38 +625,25 @@ public class RegionReplicaReplicationEndpoint extends HBaseReplicationEndpoint {
       this.initialEncodedRegionName = regionInfo.getEncodedNameAsBytes();
     }
 
-    @Override
-    public ReplicateWALEntryResponse call(int timeout) throws IOException {
-      return replayToServer(this.entries, timeout);
-    }
-
-    private ReplicateWALEntryResponse replayToServer(List<Entry> entries, int timeout)
-        throws IOException {
-      // check whether we should still replay this entry. If the regions are changed, or the
+    public ReplicateWALEntryResponse call(PayloadCarryingRpcController controller) throws Exception {
+      // Check whether we should still replay this entry. If the regions are changed, or the
       // entry is not coming form the primary region, filter it out because we do not need it.
       // Regions can change because of (1) region split (2) region merge (3) table recreated
       boolean skip = false;
-
       if (!Bytes.equals(location.getRegionInfo().getEncodedNameAsBytes(),
-        initialEncodedRegionName)) {
+          initialEncodedRegionName)) {
         skip = true;
       }
-      if (!entries.isEmpty() && !skip) {
-        Entry[] entriesArray = new Entry[entries.size()];
-        entriesArray = entries.toArray(entriesArray);
+      if (!this.entries.isEmpty() && !skip) {
+        Entry[] entriesArray = new Entry[this.entries.size()];
+        entriesArray = this.entries.toArray(entriesArray);
 
         // set the region name for the target region replica
         Pair<AdminProtos.ReplicateWALEntryRequest, CellScanner> p =
             ReplicationProtbufUtil.buildReplicateWALEntryRequest(entriesArray, location
                 .getRegionInfo().getEncodedNameAsBytes(), null, null, null);
-        try {
-          PayloadCarryingRpcController controller = rpcControllerFactory.newController(p.getSecond());
-          controller.setCallTimeout(timeout);
-          controller.setPriority(tableName);
-          return stub.replay(controller, p.getFirst());
-        } catch (ServiceException se) {
-          throw ProtobufUtil.getRemoteException(se);
-        }
+        controller.setCellScanner(p.getSecond());
+        return stub.replay(controller, p.getFirst());
       }
 
       if (skip) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/ed87a81b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/Merge.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/Merge.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/Merge.java
index d708edc..3c81cfe 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/Merge.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/Merge.java
@@ -23,19 +23,18 @@ import java.io.IOException;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.MetaTableAccessor;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.MasterNotRunningException;
+import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
@@ -80,13 +79,11 @@ public class Merge extends Configured implements Tool {
     // Verify HBase is down
     LOG.info("Verifying that HBase is not running...");
     try {
-      HBaseAdmin.checkHBaseAvailable(getConf());
+      HBaseAdmin.available(getConf());
       LOG.fatal("HBase cluster must be off-line, and is not. Aborting.");
       return -1;
     } catch (ZooKeeperConnectionException zkce) {
       // If no zk, presume no master.
-    } catch (MasterNotRunningException e) {
-      // Expected. Ignore.
     }
 
     // Initialize MetaUtils and and get the root of the HBase installation

http://git-wip-us.apache.org/repos/asf/hbase/blob/ed87a81b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java
index d778fa9..2dca6b1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java
@@ -60,7 +60,6 @@ public class TestNamespace {
   private static ZKNamespaceManager zkNamespaceManager;
   private String prefix = "TestNamespace";
 
-
   @BeforeClass
   public static void setUp() throws Exception {
     TEST_UTIL = new HBaseTestingUtility();
@@ -301,7 +300,8 @@ public class TestNamespace {
     runWithExpectedException(new Callable<Void>() {
       @Override
       public Void call() throws Exception {
-        HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("non_existing_namespace", "table1"));
+        HTableDescriptor htd =
+            new HTableDescriptor(TableName.valueOf("non_existing_namespace", "table1"));
         htd.addFamily(new HColumnDescriptor("family1"));
         admin.createTable(htd);
         return null;
@@ -387,5 +387,4 @@ public class TestNamespace {
     }
     fail("Should have thrown exception " + exceptionClass);
   }
-
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/ed87a81b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
index d088fc4..1716622 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
@@ -37,7 +37,6 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.MasterNotRunningException;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.NotServingRegionException;
 import org.apache.hadoop.hbase.ProcedureInfo;
@@ -59,7 +58,6 @@ import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
-import org.apache.hadoop.hbase.wal.FSHLogProvider;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -67,8 +65,6 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import com.google.protobuf.ServiceException;
-
 
 /**
  * Class to test HBaseAdmin.
@@ -643,11 +639,9 @@ public class TestAdmin2 {
 
     long start = System.currentTimeMillis();
     try {
-      HBaseAdmin.checkHBaseAvailable(conf);
+      HBaseAdmin.available(conf);
       assertTrue(false);
-    } catch (MasterNotRunningException ignored) {
     } catch (ZooKeeperConnectionException ignored) {
-    } catch (ServiceException ignored) {
     } catch (IOException ignored) {
     }
     long end = System.currentTimeMillis();

http://git-wip-us.apache.org/repos/asf/hbase/blob/ed87a81b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java
index 679d9c9..f49c558 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java
@@ -28,13 +28,10 @@ import java.net.UnknownHostException;
 import java.util.Random;
 import java.util.concurrent.atomic.AtomicInteger;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.MasterNotRunningException;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
 import org.apache.hadoop.hbase.ipc.AbstractRpcClient;
@@ -56,7 +53,6 @@ import com.google.protobuf.ServiceException;
 
 @Category({MediumTests.class, ClientTests.class})
 public class TestClientTimeouts {
-  private static final Log LOG = LogFactory.getLog(TestClientTimeouts.class);
   private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
   protected static int SLAVES = 1;
 
@@ -87,7 +83,6 @@ public class TestClientTimeouts {
    */
   @Test
   public void testAdminTimeout() throws Exception {
-    Connection lastConnection = null;
     boolean lastFailed = false;
     int initialInvocations = RandomTimeoutBlockingRpcChannel.invokations.get();
     RandomTimeoutRpcClient rpcClient = (RandomTimeoutRpcClient) RpcClientFactory
@@ -105,7 +100,7 @@ public class TestClientTimeouts {
           connection = ConnectionFactory.createConnection(conf);
           admin = connection.getAdmin();
           // run some admin commands
-          HBaseAdmin.checkHBaseAvailable(conf);
+          HBaseAdmin.available(conf);
           admin.setBalancerRunning(false, false);
         } catch (ZooKeeperConnectionException ex) {
           // Since we are randomly throwing SocketTimeoutExceptions, it is possible to get

http://git-wip-us.apache.org/repos/asf/hbase/blob/ed87a81b/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 1b20b76..33af5de 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
@@ -61,6 +61,7 @@ import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.exceptions.RegionMovedException;
 import org.apache.hadoop.hbase.filter.Filter;
 import org.apache.hadoop.hbase.filter.FilterBase;
+import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
 import org.apache.hadoop.hbase.ipc.RpcClient;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.master.HMaster;
@@ -103,8 +104,6 @@ public class TestHCM {
       TableName.valueOf("test2");
   private static final TableName TABLE_NAME3 =
       TableName.valueOf("test3");
-  private static final TableName TABLE_NAME4 =
-      TableName.valueOf("test4");
   private static final byte[] FAM_NAM = Bytes.toBytes("f");
   private static final byte[] ROW = Bytes.toBytes("bbb");
   private static final byte[] ROW_X = Bytes.toBytes("xxx");
@@ -407,10 +406,11 @@ public class TestHCM {
     long pauseTime;
     long baseTime = 100;
     TableName tableName = TableName.valueOf("HCM-testCallableSleep");
-    Table table = TEST_UTIL.createTable(tableName, FAM_NAM);
     RegionServerCallable<Object> regionServerCallable = new RegionServerCallable<Object>(
-        TEST_UTIL.getConnection(), tableName, ROW) {
-      public Object call(int timeout) throws IOException {
+        TEST_UTIL.getConnection(), new RpcControllerFactory(TEST_UTIL.getConfiguration()),
+        tableName, ROW) {
+      @Override
+      protected Object call(PayloadCarryingRpcController controller) throws Exception {
         return null;
       }
     };
@@ -424,9 +424,10 @@ public class TestHCM {
 
     RegionAdminServiceCallable<Object> regionAdminServiceCallable =
         new RegionAdminServiceCallable<Object>(
-        (ClusterConnection) TEST_UTIL.getConnection(), new RpcControllerFactory(
-            TEST_UTIL.getConfiguration()), tableName, ROW) {
-      public Object call(int timeout) throws IOException {
+        (ClusterConnection) TEST_UTIL.getConnection(),
+          new RpcControllerFactory(TEST_UTIL.getConfiguration()), tableName, ROW) {
+      @Override
+      public Object call(PayloadCarryingRpcController controller) throws Exception {
         return null;
       }
     };
@@ -438,16 +439,21 @@ public class TestHCM {
       assertTrue(pauseTime <= (baseTime * HConstants.RETRY_BACKOFF[i] * 1.01f));
     }
 
-    MasterCallable masterCallable = new MasterCallable(TEST_UTIL.getConnection()) {
-      public Object call(int timeout) throws IOException {
+    MasterCallable<Object> masterCallable = new MasterCallable<Object>(TEST_UTIL.getConnection(),
+        new RpcControllerFactory(TEST_UTIL.getConfiguration())) {
+      @Override
+      protected Object call(PayloadCarryingRpcController rpcController) throws Exception {
         return null;
       }
     };
-
-    for (int i = 0; i < HConstants.RETRY_BACKOFF.length; i++) {
-      pauseTime = masterCallable.sleep(baseTime, i);
-      assertTrue(pauseTime >= (baseTime * HConstants.RETRY_BACKOFF[i]));
-      assertTrue(pauseTime <= (baseTime * HConstants.RETRY_BACKOFF[i] * 1.01f));
+    try {
+      for (int i = 0; i < HConstants.RETRY_BACKOFF.length; i++) {
+        pauseTime = masterCallable.sleep(baseTime, i);
+        assertTrue(pauseTime >= (baseTime * HConstants.RETRY_BACKOFF[i]));
+        assertTrue(pauseTime <= (baseTime * HConstants.RETRY_BACKOFF[i] * 1.01f));
+      }
+    } finally {
+      masterCallable.close();
     }
   }
 
@@ -1149,7 +1155,6 @@ public class TestHCM {
     ManualEnvironmentEdge timeMachine = new ManualEnvironmentEdge();
     EnvironmentEdgeManager.injectEdge(timeMachine);
     try {
-      long timeBase = timeMachine.currentTime();
       long largeAmountOfTime = ANY_PAUSE * 1000;
       ConnectionImplementation.ServerErrorTracker tracker =
           new ConnectionImplementation.ServerErrorTracker(largeAmountOfTime, 100);

http://git-wip-us.apache.org/repos/asf/hbase/blob/ed87a81b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
index 354f0a8..d99d2ee 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
@@ -35,6 +35,8 @@ import org.apache.hadoop.hbase.client.replication.ReplicationAdmin;
 import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
+import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.regionserver.StorefileRefresherChore;
 import org.apache.hadoop.hbase.regionserver.TestHRegionServerBulkLoad;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
@@ -332,26 +334,27 @@ public class TestReplicaWithCluster {
 
     // bulk load HFiles
     LOG.debug("Loading test data");
-    @SuppressWarnings("deprecation")
     final ClusterConnection conn = (ClusterConnection) HTU.getAdmin().getConnection();
     table = conn.getTable(hdt.getTableName());
-    final String bulkToken = new SecureBulkLoadClient(table).prepareBulkLoad(conn);
-    RegionServerCallable<Void> callable = new RegionServerCallable<Void>(
-      conn, hdt.getTableName(), TestHRegionServerBulkLoad.rowkey(0)) {
-        @Override
-        public Void call(int timeout) throws Exception {
-          LOG.debug("Going to connect to server " + getLocation() + " for row "
+    final String bulkToken =
+        new SecureBulkLoadClient(HTU.getConfiguration(), table).prepareBulkLoad(conn);
+    RegionServerCallable<Void> callable = new RegionServerCallable<Void>(conn,
+        new RpcControllerFactory(HTU.getConfiguration()), hdt.getTableName(),
+        TestHRegionServerBulkLoad.rowkey(0)) {
+      @Override
+      protected Void call(PayloadCarryingRpcController controller) throws Exception {
+        LOG.debug("Going to connect to server " + getLocation() + " for row "
             + Bytes.toStringBinary(getRow()));
-          SecureBulkLoadClient secureClient = null;
-          byte[] regionName = getLocation().getRegionInfo().getRegionName();
-          try (Table table = conn.getTable(getTableName())) {
-            secureClient = new SecureBulkLoadClient(table);
-            secureClient.secureBulkLoadHFiles(getStub(), famPaths, regionName,
-                  true, null, bulkToken);
-          }
-          return null;
+        SecureBulkLoadClient secureClient = null;
+        byte[] regionName = getLocation().getRegionInfo().getRegionName();
+        try (Table table = conn.getTable(getTableName())) {
+          secureClient = new SecureBulkLoadClient(HTU.getConfiguration(), table);
+          secureClient.secureBulkLoadHFiles(getStub(), famPaths, regionName,
+              true, null, bulkToken);
         }
-      };
+        return null;
+      }
+    };
     RpcRetryingCallerFactory factory = new RpcRetryingCallerFactory(HTU.getConfiguration());
     RpcRetryingCaller<Void> caller = factory.newCaller();
     caller.callWithRetries(callable, 10000);

http://git-wip-us.apache.org/repos/asf/hbase/blob/ed87a81b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java
index 6e68201..30805c0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java
@@ -62,6 +62,8 @@ import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.io.hfile.HFileContext;
 import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
+import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
+import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionRequest;
@@ -198,19 +200,20 @@ public class TestHRegionServerBulkLoad {
       }
 
       // bulk load HFiles
-      final ClusterConnection conn = (ClusterConnection) UTIL.getAdmin().getConnection();
+      final ClusterConnection conn = (ClusterConnection)UTIL.getConnection();
       Table table = conn.getTable(tableName);
-      final String bulkToken = new SecureBulkLoadClient(table).prepareBulkLoad(conn);
-      RegionServerCallable<Void> callable =
-          new RegionServerCallable<Void>(conn, tableName, Bytes.toBytes("aaa")) {
+      final String bulkToken = new SecureBulkLoadClient(UTIL.getConfiguration(), table).
+          prepareBulkLoad(conn);
+      RegionServerCallable<Void> callable = new RegionServerCallable<Void>(conn,
+          new RpcControllerFactory(UTIL.getConfiguration()), tableName, Bytes.toBytes("aaa")) {
         @Override
-        public Void call(int callTimeout) throws Exception {
+        public Void call(PayloadCarryingRpcController controller) throws Exception {
           LOG.debug("Going to connect to server " + getLocation() + " for row "
               + Bytes.toStringBinary(getRow()));
           SecureBulkLoadClient secureClient = null;
           byte[] regionName = getLocation().getRegionInfo().getRegionName();
           try (Table table = conn.getTable(getTableName())) {
-            secureClient = new SecureBulkLoadClient(table);
+            secureClient = new SecureBulkLoadClient(UTIL.getConfiguration(), table);
             secureClient.secureBulkLoadHFiles(getStub(), famPaths, regionName,
                   true, null, bulkToken);
           }
@@ -224,15 +227,15 @@ public class TestHRegionServerBulkLoad {
       // Periodically do compaction to reduce the number of open file handles.
       if (numBulkLoads.get() % 5 == 0) {
         // 5 * 50 = 250 open file handles!
-        callable = new RegionServerCallable<Void>(conn, tableName, Bytes.toBytes("aaa")) {
+        callable = new RegionServerCallable<Void>(conn,
+            new RpcControllerFactory(UTIL.getConfiguration()), tableName, Bytes.toBytes("aaa")) {
           @Override
-          public Void call(int callTimeout) throws Exception {
+          protected Void call(PayloadCarryingRpcController controller) throws Exception {
             LOG.debug("compacting " + getLocation() + " for row "
                 + Bytes.toStringBinary(getRow()));
             AdminProtos.AdminService.BlockingInterface server =
               conn.getAdmin(getLocation().getServerName());
-            CompactRegionRequest request =
-              RequestConverter.buildCompactRegionRequest(
+            CompactRegionRequest request = RequestConverter.buildCompactRegionRequest(
                 getLocation().getRegionInfo().getRegionName(), true, null);
             server.compactRegion(null, request);
             numCompactions.incrementAndGet();

http://git-wip-us.apache.org/repos/asf/hbase/blob/ed87a81b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoadWithOldClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoadWithOldClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoadWithOldClient.java
index d55adef..7560a41 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoadWithOldClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoadWithOldClient.java
@@ -33,6 +33,8 @@ import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.RegionServerCallable;
 import org.apache.hadoop.hbase.client.RpcRetryingCaller;
 import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
+import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
+import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionRequest;
@@ -89,10 +91,12 @@ public class TestHRegionServerBulkLoadWithOldClient extends TestHRegionServerBul
 
       // bulk load HFiles
       final ClusterConnection conn = (ClusterConnection) UTIL.getAdmin().getConnection();
+      RpcControllerFactory rpcControllerFactory = new RpcControllerFactory(UTIL.getConfiguration());
       RegionServerCallable<Void> callable =
-          new RegionServerCallable<Void>(conn, tableName, Bytes.toBytes("aaa")) {
+          new RegionServerCallable<Void>(conn, rpcControllerFactory, tableName,
+              Bytes.toBytes("aaa")) {
         @Override
-        public Void call(int callTimeout) throws Exception {
+        protected Void call(PayloadCarryingRpcController controller) throws Exception {
           LOG.info("Non-secure old client");
           byte[] regionName = getLocation().getRegionInfo().getRegionName();
               BulkLoadHFileRequest request =
@@ -109,9 +113,10 @@ public class TestHRegionServerBulkLoadWithOldClient extends TestHRegionServerBul
       // Periodically do compaction to reduce the number of open file handles.
       if (numBulkLoads.get() % 5 == 0) {
         // 5 * 50 = 250 open file handles!
-        callable = new RegionServerCallable<Void>(conn, tableName, Bytes.toBytes("aaa")) {
+        callable = new RegionServerCallable<Void>(conn, rpcControllerFactory, tableName,
+            Bytes.toBytes("aaa")) {
           @Override
-          public Void call(int callTimeout) throws Exception {
+          protected Void call(PayloadCarryingRpcController controller) throws Exception {
             LOG.debug("compacting " + getLocation() + " for row "
                 + Bytes.toStringBinary(getRow()));
             AdminProtos.AdminService.BlockingInterface server =

http://git-wip-us.apache.org/repos/asf/hbase/blob/ed87a81b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoadWithOldSecureEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoadWithOldSecureEndpoint.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoadWithOldSecureEndpoint.java
index 6de6261..0bc9498 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoadWithOldSecureEndpoint.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoadWithOldSecureEndpoint.java
@@ -33,13 +33,13 @@ import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.RegionServerCallable;
 import org.apache.hadoop.hbase.client.RpcRetryingCaller;
 import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
-import org.apache.hadoop.hbase.client.SecureBulkLoadClient;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
+import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
+import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionRequest;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequest;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -62,7 +62,8 @@ public class TestHRegionServerBulkLoadWithOldSecureEndpoint extends TestHRegionS
     super(duration);
   }
 
-  private static final Log LOG = LogFactory.getLog(TestHRegionServerBulkLoadWithOldSecureEndpoint.class);
+  private static final Log LOG =
+      LogFactory.getLog(TestHRegionServerBulkLoadWithOldSecureEndpoint.class);
 
   @BeforeClass
   public static void setUpBeforeClass() throws IOException {
@@ -103,16 +104,17 @@ public class TestHRegionServerBulkLoadWithOldSecureEndpoint extends TestHRegionS
       final ClusterConnection conn = (ClusterConnection) UTIL.getAdmin().getConnection();
       Table table = conn.getTable(tableName);
       final String bulkToken = new SecureBulkLoadEndpointClient(table).prepareBulkLoad(tableName);
+      RpcControllerFactory rpcControllerFactory = new RpcControllerFactory(UTIL.getConfiguration());
       RegionServerCallable<Void> callable =
-          new RegionServerCallable<Void>(conn, tableName, Bytes.toBytes("aaa")) {
+          new RegionServerCallable<Void>(conn, rpcControllerFactory, tableName,
+              Bytes.toBytes("aaa")) {
             @Override
-            public Void call(int callTimeout) throws Exception {
-              LOG.debug("Going to connect to server " + getLocation() + " for row "
-                  + Bytes.toStringBinary(getRow()));
+            protected Void call(PayloadCarryingRpcController controller) throws Exception {
+              LOG.debug("Going to connect to server " + getLocation() + " for row " +
+                  Bytes.toStringBinary(getRow()));
               try (Table table = conn.getTable(getTableName())) {
-                boolean loaded =
-                    new SecureBulkLoadEndpointClient(table).bulkLoadHFiles(famPaths, null,
-                      bulkToken, getLocation().getRegionInfo().getStartKey());
+                boolean loaded = new SecureBulkLoadEndpointClient(table).bulkLoadHFiles(famPaths,
+                    null, bulkToken, getLocation().getRegionInfo().getStartKey());
               }
               return null;
             }
@@ -124,9 +126,10 @@ public class TestHRegionServerBulkLoadWithOldSecureEndpoint extends TestHRegionS
       // Periodically do compaction to reduce the number of open file handles.
       if (numBulkLoads.get() % 5 == 0) {
         // 5 * 50 = 250 open file handles!
-        callable = new RegionServerCallable<Void>(conn, tableName, Bytes.toBytes("aaa")) {
+        callable = new RegionServerCallable<Void>(conn, rpcControllerFactory, tableName,
+            Bytes.toBytes("aaa")) {
           @Override
-          public Void call(int callTimeout) throws Exception {
+          protected Void call(PayloadCarryingRpcController controller) throws Exception {
             LOG.debug("compacting " + getLocation() + " for row "
                 + Bytes.toStringBinary(getRow()));
             AdminProtos.AdminService.BlockingInterface server =

http://git-wip-us.apache.org/repos/asf/hbase/blob/ed87a81b/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/SparkSQLPushDownFilter.java
----------------------------------------------------------------------
diff --git a/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/SparkSQLPushDownFilter.java b/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/SparkSQLPushDownFilter.java
index fa66d69..3e90fe1 100644
--- a/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/SparkSQLPushDownFilter.java
+++ b/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/SparkSQLPushDownFilter.java
@@ -17,8 +17,6 @@
 
 package org.apache.hadoop.hbase.spark;
 
-import com.google.protobuf.ByteString;
-import com.google.protobuf.InvalidProtocolBufferException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.Cell;
@@ -37,6 +35,8 @@ import java.io.IOException;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import com.google.protobuf.InvalidProtocolBufferException;
+import com.google.protobuf.ByteString;
 
 /**
  * This filter will push down all qualifier logic given to us


[25/50] [abbrv] hbase git commit: HBASE-15866 Split hbase.rpc.timeout into *.read.timeout and *.write.timeout

Posted by sy...@apache.org.
HBASE-15866 Split hbase.rpc.timeout into *.read.timeout and *.write.timeout

Signed-off-by: Andrew Purtell <ap...@apache.org>
Amending-Author: Andrew Purtell <ap...@apache.org>


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

Branch: refs/heads/hbase-12439
Commit: 30d7eeaefe431bc263519064662e6dc8ad8ff05a
Parents: 4e08a8b
Author: Vivek <vk...@salesforce.com>
Authored: Fri Aug 5 17:25:06 2016 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Sat Aug 6 16:55:09 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/client/AsyncProcess.java       |  5 +-
 .../hbase/client/BufferedMutatorImpl.java       |  8 +-
 .../hbase/client/ConnectionImplementation.java  |  3 +-
 .../org/apache/hadoop/hbase/client/HTable.java  | 58 +++++++++----
 .../hadoop/hbase/client/HTableMultiplexer.java  |  6 +-
 .../org/apache/hadoop/hbase/client/Table.java   | 43 +++++++++-
 .../hadoop/hbase/client/TestAsyncProcess.java   | 11 ++-
 .../org/apache/hadoop/hbase/HConstants.java     | 13 +++
 .../hadoop/hbase/rest/client/RemoteHTable.java  | 22 +++++
 .../hadoop/hbase/client/HTableWrapper.java      | 14 +++
 .../hbase/client/HConnectionTestingUtility.java |  6 +-
 .../org/apache/hadoop/hbase/client/TestHCM.java | 90 ++++++++++++++++++--
 .../hbase/regionserver/RegionAsTable.java       | 14 +++
 13 files changed, 257 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/30d7eeae/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
index 4514560..1383ca0 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
@@ -281,7 +281,7 @@ class AsyncProcess {
 
   public AsyncProcess(ClusterConnection hc, Configuration conf, ExecutorService pool,
       RpcRetryingCallerFactory rpcCaller, boolean useGlobalErrors,
-      RpcControllerFactory rpcFactory) {
+      RpcControllerFactory rpcFactory, int rpcTimeout) {
     if (hc == null) {
       throw new IllegalArgumentException("ClusterConnection cannot be null.");
     }
@@ -297,8 +297,7 @@ class AsyncProcess {
     // how many times we could try in total, one more than retry number
     this.numTries = conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
         HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER) + 1;
-    this.timeout = conf.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY,
-        HConstants.DEFAULT_HBASE_RPC_TIMEOUT);
+    this.timeout = rpcTimeout;
     this.primaryCallTimeoutMicroseconds = conf.getInt(PRIMARY_CALL_TIMEOUT_KEY, 10000);
 
     this.maxTotalConcurrentTasks = conf.getInt(HConstants.HBASE_CLIENT_MAX_TOTAL_TASKS,

http://git-wip-us.apache.org/repos/asf/hbase/blob/30d7eeae/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java
index e98ad4e..39e4f75 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java
@@ -19,6 +19,7 @@ import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants; // Needed for write rpc timeout
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
@@ -72,6 +73,7 @@ public class BufferedMutatorImpl implements BufferedMutator {
   private final int maxKeyValueSize;
   private boolean closed = false;
   private final ExecutorService pool;
+  private int writeRpcTimeout; // needed to pass in through AsyncProcess constructor
 
   @VisibleForTesting
   protected AsyncProcess ap; // non-final so can be overridden in test
@@ -94,8 +96,12 @@ public class BufferedMutatorImpl implements BufferedMutator {
     this.maxKeyValueSize = params.getMaxKeyValueSize() != BufferedMutatorParams.UNSET ?
         params.getMaxKeyValueSize() : tableConf.getMaxKeyValueSize();
 
+    this.writeRpcTimeout = conn.getConfiguration().getInt(HConstants.HBASE_RPC_WRITE_TIMEOUT_KEY,
+        conn.getConfiguration().getInt(HConstants.HBASE_RPC_TIMEOUT_KEY,
+            HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
+
     // puts need to track errors globally due to how the APIs currently work.
-    ap = new AsyncProcess(connection, conf, pool, rpcCallerFactory, true, rpcFactory);
+    ap = new AsyncProcess(connection, conf, pool, rpcCallerFactory, true, rpcFactory, writeRpcTimeout);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/30d7eeae/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
index 8dcda13..04edd25 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
@@ -1823,7 +1823,8 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
   // For tests to override.
   protected AsyncProcess createAsyncProcess(Configuration conf) {
     // No default pool available.
-    return new AsyncProcess(this, conf, batchPool, rpcCallerFactory, false, rpcControllerFactory);
+    int rpcTimeout = conf.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT);
+    return new AsyncProcess(this, conf, batchPool, rpcCallerFactory, false, rpcControllerFactory, rpcTimeout);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/30d7eeae/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 fbd9f51..882e21b 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
@@ -112,7 +112,8 @@ public class HTable implements Table {
   protected long scannerMaxResultSize;
   private ExecutorService pool;  // For Multi & Scan
   private int operationTimeout; // global timeout for each blocking method with retrying rpc
-  private int rpcTimeout; // timeout for each rpc request
+  private int readRpcTimeout; // timeout for each read rpc request
+  private int writeRpcTimeout; // timeout for each write 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;
@@ -212,8 +213,12 @@ public class HTable implements Table {
 
     this.operationTimeout = tableName.isSystemTable() ?
         connConfiguration.getMetaOperationTimeout() : connConfiguration.getOperationTimeout();
-    this.rpcTimeout = configuration.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY,
-        HConstants.DEFAULT_HBASE_RPC_TIMEOUT);
+    this.readRpcTimeout = configuration.getInt(HConstants.HBASE_RPC_READ_TIMEOUT_KEY,
+        configuration.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY,
+            HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
+    this.writeRpcTimeout = configuration.getInt(HConstants.HBASE_RPC_WRITE_TIMEOUT_KEY,
+        configuration.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY,
+            HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
     this.scannerCaching = connConfiguration.getScannerCaching();
     this.scannerMaxResultSize = connConfiguration.getScannerMaxResultSize();
     if (this.rpcCallerFactory == null) {
@@ -257,7 +262,7 @@ public class HTable implements Table {
   @Override
   public HTableDescriptor getTableDescriptor() throws IOException {
     HTableDescriptor htd = HBaseAdmin.getTableDescriptor(tableName, connection, rpcCallerFactory,
-      rpcControllerFactory, operationTimeout, rpcTimeout);
+      rpcControllerFactory, operationTimeout, readRpcTimeout);
     if (htd != null) {
       return new UnmodifyableHTableDescriptor(htd);
     }
@@ -430,7 +435,7 @@ public class HTable implements Table {
           }
         }
       };
-      return rpcCallerFactory.<Result>newCaller(rpcTimeout).callWithRetries(callable,
+      return rpcCallerFactory.<Result>newCaller(readRpcTimeout).callWithRetries(callable,
           this.operationTimeout);
     }
 
@@ -528,7 +533,7 @@ public class HTable implements Table {
         }
       }
     };
-    rpcCallerFactory.<Boolean> newCaller(rpcTimeout).callWithRetries(callable,
+    rpcCallerFactory.<Boolean> newCaller(writeRpcTimeout).callWithRetries(callable,
         this.operationTimeout);
   }
 
@@ -654,7 +659,7 @@ public class HTable implements Table {
           }
         }
       };
-    return rpcCallerFactory.<Result> newCaller(rpcTimeout).callWithRetries(callable,
+    return rpcCallerFactory.<Result> newCaller(writeRpcTimeout).callWithRetries(callable,
         this.operationTimeout);
   }
 
@@ -686,7 +691,7 @@ public class HTable implements Table {
         }
       }
     };
-    return rpcCallerFactory.<Result> newCaller(rpcTimeout).callWithRetries(callable,
+    return rpcCallerFactory.<Result> newCaller(writeRpcTimeout).callWithRetries(callable,
         this.operationTimeout);
   }
 
@@ -742,7 +747,7 @@ public class HTable implements Table {
           }
         }
       };
-    return rpcCallerFactory.<Long> newCaller(rpcTimeout).callWithRetries(callable,
+    return rpcCallerFactory.<Long> newCaller(writeRpcTimeout).callWithRetries(callable,
         this.operationTimeout);
   }
 
@@ -772,7 +777,7 @@ public class HTable implements Table {
           }
         }
       };
-    return rpcCallerFactory.<Boolean> newCaller(rpcTimeout).callWithRetries(callable,
+    return rpcCallerFactory.<Boolean> newCaller(writeRpcTimeout).callWithRetries(callable,
         this.operationTimeout);
   }
 
@@ -803,7 +808,7 @@ public class HTable implements Table {
           }
         }
       };
-    return rpcCallerFactory.<Boolean> newCaller(rpcTimeout).callWithRetries(callable,
+    return rpcCallerFactory.<Boolean> newCaller(writeRpcTimeout).callWithRetries(callable,
         this.operationTimeout);
   }
 
@@ -833,7 +838,7 @@ public class HTable implements Table {
           }
         }
       };
-    return rpcCallerFactory.<Boolean> newCaller(rpcTimeout).callWithRetries(callable,
+    return rpcCallerFactory.<Boolean> newCaller(writeRpcTimeout).callWithRetries(callable,
         this.operationTimeout);
   }
 
@@ -864,7 +869,7 @@ public class HTable implements Table {
           }
         }
       };
-    return rpcCallerFactory.<Boolean> newCaller(rpcTimeout).callWithRetries(callable,
+    return rpcCallerFactory.<Boolean> newCaller(writeRpcTimeout).callWithRetries(callable,
         this.operationTimeout);
   }
 
@@ -1196,13 +1201,34 @@ public class HTable implements Table {
   }
 
   @Override
+  @Deprecated
   public int getRpcTimeout() {
-    return rpcTimeout;
+    return readRpcTimeout;
   }
 
   @Override
+  @Deprecated
   public void setRpcTimeout(int rpcTimeout) {
-    this.rpcTimeout = rpcTimeout;
+    this.readRpcTimeout = rpcTimeout;
+    this.writeRpcTimeout = rpcTimeout;
+  }
+
+  @Override
+  public int getWriteRpcTimeout() {
+    return writeRpcTimeout;
+  }
+
+  @Override
+  public void setWriteRpcTimeout(int writeRpcTimeout) {
+    this.writeRpcTimeout = writeRpcTimeout;
+  }
+
+  @Override
+  public int getReadRpcTimeout() { return readRpcTimeout; }
+
+  @Override
+  public void setReadRpcTimeout(int readRpcTimeout) {
+    this.readRpcTimeout = readRpcTimeout;
   }
 
   @Override
@@ -1282,7 +1308,7 @@ public class HTable implements Table {
     AsyncProcess asyncProcess =
         new AsyncProcess(connection, configuration, pool,
             RpcRetryingCallerFactory.instantiate(configuration, connection.getStatisticsTracker()),
-            true, RpcControllerFactory.instantiate(configuration));
+            true, RpcControllerFactory.instantiate(configuration), readRpcTimeout);
 
     AsyncRequestFuture future = asyncProcess.submitAll(tableName, execs,
         new Callback<ClientProtos.CoprocessorServiceResult>() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/30d7eeae/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java
index f1bbcb3..ba963c2 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java
@@ -442,6 +442,7 @@ public class HTableMultiplexer {
     private final ScheduledExecutorService executor;
     private final int maxRetryInQueue;
     private final AtomicInteger retryInQueue = new AtomicInteger(0);
+    private final int writeRpcTimeout; // needed to pass in through AsyncProcess constructor
 
     public FlushWorker(Configuration conf, ClusterConnection conn, HRegionLocation addr,
         HTableMultiplexer htableMultiplexer, int perRegionServerBufferQueueSize,
@@ -451,7 +452,10 @@ public class HTableMultiplexer {
       this.queue = new LinkedBlockingQueue<>(perRegionServerBufferQueueSize);
       RpcRetryingCallerFactory rpcCallerFactory = RpcRetryingCallerFactory.instantiate(conf);
       RpcControllerFactory rpcControllerFactory = RpcControllerFactory.instantiate(conf);
-      this.ap = new AsyncProcess(conn, conf, pool, rpcCallerFactory, false, rpcControllerFactory);
+      this.writeRpcTimeout = conf.getInt(HConstants.HBASE_RPC_WRITE_TIMEOUT_KEY,
+          conf.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY,
+              HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
+      this.ap = new AsyncProcess(conn, conf, pool, rpcCallerFactory, false, rpcControllerFactory, writeRpcTimeout);
       this.executor = executor;
       this.maxRetryInQueue = conf.getInt(TABLE_MULTIPLEXER_MAX_RETRIES_IN_QUEUE, 10000);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/30d7eeae/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 f2cec97..4d93442 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
@@ -585,17 +585,56 @@ public interface Table extends Closeable {
   int getOperationTimeout();
 
   /**
+   * Get timeout (millisecond) of each rpc request in this Table instance.
+   *
+   * @returns Currently configured read timeout
+   * @deprecated Use getReadRpcTimeout or getWriteRpcTimeout instead
+   */
+  @Deprecated
+  int getRpcTimeout();
+
+  /**
    * 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.
+   * <p>
+   * NOTE: This will set both the read and write timeout settings to the provided value.
+   *
    * @param rpcTimeout the timeout of each rpc request in millisecond.
+   *
+   * @deprecated Use setReadRpcTimeout or setWriteRpcTimeout instead
    */
+  @Deprecated
   void setRpcTimeout(int rpcTimeout);
 
   /**
-   * Get timeout (millisecond) of each rpc request in this Table instance.
+   * Get timeout (millisecond) of each rpc read request in this Table instance.
    */
-  int getRpcTimeout();
+  int getReadRpcTimeout();
+
+  /**
+   * Set timeout (millisecond) of each rpc read request in operations of this Table instance, will
+   * override the value of hbase.rpc.read.timeout in configuration.
+   * If a rpc read request waiting too long, it will stop waiting and send a new request to retry
+   * until retries exhausted or operation timeout reached.
+   *
+   * @param readRpcTimeout
+   */
+  void setReadRpcTimeout(int readRpcTimeout);
 
+  /**
+   * Get timeout (millisecond) of each rpc write request in this Table instance.
+   */
+  int getWriteRpcTimeout();
+
+  /**
+   * Set timeout (millisecond) of each rpc write request in operations of this Table instance, will
+   * override the value of hbase.rpc.write.timeout in configuration.
+   * If a rpc write request waiting too long, it will stop waiting and send a new request to retry
+   * until retries exhausted or operation timeout reached.
+   *
+   * @param writeRpcTimeout
+   */
+  void setWriteRpcTimeout(int writeRpcTimeout);
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/30d7eeae/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
index d943316..0aa9704 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
@@ -138,6 +138,7 @@ public class TestAsyncProcess {
     final AtomicInteger nbActions = new AtomicInteger();
     public List<AsyncRequestFuture> allReqs = new ArrayList<AsyncRequestFuture>();
     public AtomicInteger callsCt = new AtomicInteger();
+    private static int rpcTimeout = conf.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT);
 
     @Override
     protected <Res> AsyncRequestFutureImpl<Res> createAsyncRequestFuture(TableName tableName,
@@ -157,14 +158,14 @@ public class TestAsyncProcess {
     public MyAsyncProcess(ClusterConnection hc, Configuration conf, AtomicInteger nbThreads) {
       super(hc, conf, new ThreadPoolExecutor(1, 20, 60, TimeUnit.SECONDS,
           new SynchronousQueue<Runnable>(), new CountingThreadFactory(nbThreads)),
-            new RpcRetryingCallerFactory(conf), false, new RpcControllerFactory(conf));
+            new RpcRetryingCallerFactory(conf), false, new RpcControllerFactory(conf), rpcTimeout);
     }
 
     public MyAsyncProcess(
         ClusterConnection hc, Configuration conf, boolean useGlobalErrors) {
       super(hc, conf, new ThreadPoolExecutor(1, 20, 60, TimeUnit.SECONDS,
         new SynchronousQueue<Runnable>(), new CountingThreadFactory(new AtomicInteger())),
-          new RpcRetryingCallerFactory(conf), useGlobalErrors, new RpcControllerFactory(conf));
+          new RpcRetryingCallerFactory(conf), useGlobalErrors, new RpcControllerFactory(conf), rpcTimeout);
     }
 
     public MyAsyncProcess(ClusterConnection hc, Configuration conf, boolean useGlobalErrors,
@@ -176,7 +177,7 @@ public class TestAsyncProcess {
           throw new RejectedExecutionException("test under failure");
         }
       },
-          new RpcRetryingCallerFactory(conf), useGlobalErrors, new RpcControllerFactory(conf));
+          new RpcRetryingCallerFactory(conf), useGlobalErrors, new RpcControllerFactory(conf), rpcTimeout);
     }
 
     @Override
@@ -1111,10 +1112,12 @@ public class TestAsyncProcess {
   }
 
   static class AsyncProcessForThrowableCheck extends AsyncProcess {
+    private static int rpcTimeout = conf.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT);
+
     public AsyncProcessForThrowableCheck(ClusterConnection hc, Configuration conf,
         ExecutorService pool) {
       super(hc, conf, pool, new RpcRetryingCallerFactory(conf), false, new RpcControllerFactory(
-          conf));
+          conf), rpcTimeout);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/30d7eeae/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 256c374..ce18ef5 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
@@ -815,10 +815,23 @@ public final class HConstants {
 
   /**
    * timeout for each RPC
+   * @deprecated Use {@link #HBASE_RPC_READ_TIMEOUT_KEY} or {@link #HBASE_RPC_WRITE_TIMEOUT_KEY}
+   * instead.
    */
+  @Deprecated
   public static final String HBASE_RPC_TIMEOUT_KEY = "hbase.rpc.timeout";
 
   /**
+   * timeout for each read RPC
+   */
+  public static final String HBASE_RPC_READ_TIMEOUT_KEY = "hbase.rpc.read.timeout";
+
+  /**
+   * timeout for each write RPC
+   */
+  public static final String HBASE_RPC_WRITE_TIMEOUT_KEY = "hbase.rpc.write.timeout";
+
+  /**
    * Default value of {@link #HBASE_RPC_TIMEOUT_KEY}
    */
   public static final int DEFAULT_HBASE_RPC_TIMEOUT = 60000;

http://git-wip-us.apache.org/repos/asf/hbase/blob/30d7eeae/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 b9e393e..33c2fc2 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
@@ -860,12 +860,34 @@ public class RemoteHTable implements Table {
   }
 
   @Override
+  @Deprecated
   public void setRpcTimeout(int rpcTimeout) {
     throw new UnsupportedOperationException();
   }
 
   @Override
+  @Deprecated
   public int getRpcTimeout() {
     throw new UnsupportedOperationException();
   }
+
+  @Override
+  public int getReadRpcTimeout() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void setReadRpcTimeout(int readRpcTimeout) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int getWriteRpcTimeout() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void setWriteRpcTimeout(int writeRpcTimeout) {
+    throw new UnsupportedOperationException();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/30d7eeae/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 5da0df7..6a73261 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
@@ -308,12 +308,26 @@ public final class HTableWrapper implements Table {
   }
 
   @Override
+  @Deprecated
   public void setRpcTimeout(int rpcTimeout) {
     table.setRpcTimeout(rpcTimeout);
   }
 
   @Override
+  public void setWriteRpcTimeout(int writeRpcTimeout) { table.setWriteRpcTimeout(writeRpcTimeout); }
+
+  @Override
+  public void setReadRpcTimeout(int readRpcTimeout) { table.setReadRpcTimeout(readRpcTimeout); }
+
+  @Override
+  @Deprecated
   public int getRpcTimeout() {
     return table.getRpcTimeout();
   }
+
+  @Override
+  public int getWriteRpcTimeout() { return table.getWriteRpcTimeout(); }
+
+  @Override
+  public int getReadRpcTimeout() { return table.getReadRpcTimeout(); }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/30d7eeae/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java
index 265e3c1..036b38f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java
@@ -23,6 +23,7 @@ import java.util.concurrent.atomic.AtomicLong;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.RegionLocations;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.ServerName;
@@ -126,7 +127,8 @@ public class HConnectionTestingUtility {
     Mockito.when(c.getNonceGenerator()).thenReturn(ng);
     Mockito.when(c.getAsyncProcess()).thenReturn(
       new AsyncProcess(c, conf, null, RpcRetryingCallerFactory.instantiate(conf), false,
-          RpcControllerFactory.instantiate(conf)));
+          RpcControllerFactory.instantiate(conf), conf.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY,
+              HConstants.DEFAULT_HBASE_RPC_TIMEOUT)));
     Mockito.when(c.getNewRpcRetryingCallerFactory(conf)).thenReturn(
         RpcRetryingCallerFactory.instantiate(conf,
             RetryingCallerInterceptorFactory.NO_OP_INTERCEPTOR, null));
@@ -194,4 +196,4 @@ public class HConnectionTestingUtility {
       return result;
     }
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/30d7eeae/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 1b20b76..4d47bde 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
@@ -18,11 +18,7 @@
  */
 package org.apache.hadoop.hbase.client;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.*;
 
 import java.io.IOException;
 import java.lang.reflect.Field;
@@ -149,6 +145,16 @@ public class TestHCM {
     }
   }
 
+  public static class SleepWriteCoprocessor extends BaseRegionObserver {
+    public static final int SLEEP_TIME = 5000;
+    @Override
+    public Result preIncrement(final ObserverContext<RegionCoprocessorEnvironment> e,
+                               final Increment increment) throws IOException {
+      Threads.sleep(SLEEP_TIME);
+      return super.preIncrement(e, increment);
+    }
+  }
+
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     TEST_UTIL.getConfiguration().setBoolean(HConstants.STATUS_PUBLISHED, true);
@@ -351,7 +357,7 @@ public class TestHCM {
     }
   }
 
-  @Test(expected = RetriesExhaustedException.class)
+  @Test
   public void testRpcTimeout() throws Exception {
     HTableDescriptor hdt = TEST_UTIL.createTableDescriptor("HCM-testRpcTimeout");
     hdt.addCoprocessor(SleepCoprocessor.class.getName());
@@ -361,6 +367,78 @@ public class TestHCM {
       t.setRpcTimeout(SleepCoprocessor.SLEEP_TIME / 2);
       t.setOperationTimeout(SleepCoprocessor.SLEEP_TIME * 100);
       t.get(new Get(FAM_NAM));
+      fail("Get should not have succeeded");
+    } catch (RetriesExhaustedException e) {
+      // expected
+    }
+
+    // Again, with configuration based override
+    c.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, SleepCoprocessor.SLEEP_TIME / 2);
+    try (Connection conn = ConnectionFactory.createConnection(c)) {
+      try (Table t = conn.getTable(hdt.getTableName())) {
+        t.get(new Get(FAM_NAM));
+        fail("Get should not have succeeded");
+      } catch (RetriesExhaustedException e) {
+        // expected
+      }
+    }
+  }
+
+  @Test
+  public void testWriteRpcTimeout() throws Exception {
+    HTableDescriptor hdt = TEST_UTIL.createTableDescriptor("HCM-testWriteRpcTimeout");
+    hdt.addCoprocessor(SleepWriteCoprocessor.class.getName());
+    Configuration c = new Configuration(TEST_UTIL.getConfiguration());
+
+    try (Table t = TEST_UTIL.createTable(hdt, new byte[][] { FAM_NAM }, c)) {
+      t.setWriteRpcTimeout(SleepWriteCoprocessor.SLEEP_TIME / 2);
+      t.setOperationTimeout(SleepWriteCoprocessor.SLEEP_TIME * 100);
+      Increment i = new Increment(FAM_NAM);
+      i.addColumn(FAM_NAM, FAM_NAM, 1);
+      t.increment(i);
+      fail("Write should not have succeeded");
+    } catch (RetriesExhaustedException e) {
+      // expected
+    }
+
+    // Again, with configuration based override
+    c.setInt(HConstants.HBASE_RPC_WRITE_TIMEOUT_KEY, SleepWriteCoprocessor.SLEEP_TIME / 2);
+    try (Connection conn = ConnectionFactory.createConnection(c)) {
+      try (Table t = conn.getTable(hdt.getTableName())) {
+        Increment i = new Increment(FAM_NAM);
+        i.addColumn(FAM_NAM, FAM_NAM, 1);
+        t.increment(i);
+        fail("Write should not have succeeded");
+      } catch (RetriesExhaustedException e) {
+        // expected
+      }
+    }
+  }
+
+  @Test
+  public void testReadRpcTimeout() throws Exception {
+    HTableDescriptor hdt = TEST_UTIL.createTableDescriptor("HCM-testReadRpcTimeout");
+    hdt.addCoprocessor(SleepCoprocessor.class.getName());
+    Configuration c = new Configuration(TEST_UTIL.getConfiguration());
+
+    try (Table t = TEST_UTIL.createTable(hdt, new byte[][] { FAM_NAM }, c)) {
+      t.setReadRpcTimeout(SleepCoprocessor.SLEEP_TIME / 2);
+      t.setOperationTimeout(SleepCoprocessor.SLEEP_TIME * 100);
+      t.get(new Get(FAM_NAM));
+      fail("Get should not have succeeded");
+    } catch (RetriesExhaustedException e) {
+      // expected
+    }
+
+    // Again, with configuration based override
+    c.setInt(HConstants.HBASE_RPC_READ_TIMEOUT_KEY, SleepCoprocessor.SLEEP_TIME / 2);
+    try (Connection conn = ConnectionFactory.createConnection(c)) {
+      try (Table t = conn.getTable(hdt.getTableName())) {
+        t.get(new Get(FAM_NAM));
+        fail("Get should not have succeeded");
+      } catch (RetriesExhaustedException e) {
+        // expected
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/30d7eeae/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/RegionAsTable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/RegionAsTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/RegionAsTable.java
index 770c39b..d2e78b7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/RegionAsTable.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/RegionAsTable.java
@@ -333,12 +333,26 @@ public class RegionAsTable implements Table {
   }
 
   @Override
+  @Deprecated
   public void setRpcTimeout(int rpcTimeout) {
     throw new UnsupportedOperationException();
   }
 
   @Override
+  public void setWriteRpcTimeout(int writeRpcTimeout) {throw new UnsupportedOperationException(); }
+
+  @Override
+  public void setReadRpcTimeout(int readRpcTimeout) {throw new UnsupportedOperationException(); }
+
+  @Override
+  @Deprecated
   public int getRpcTimeout() {
     throw new UnsupportedOperationException();
   }
+
+  @Override
+  public int getWriteRpcTimeout() { throw new UnsupportedOperationException(); }
+
+  @Override
+  public int getReadRpcTimeout() { throw new UnsupportedOperationException(); }
 }
\ No newline at end of file


[22/50] [abbrv] hbase git commit: HBASE-15882 Upgrade to yetus precommit 0.3.0

Posted by sy...@apache.org.
HBASE-15882 Upgrade to yetus precommit 0.3.0

Signed-off-by: Sean Busbey <bu...@apache.org>


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

Branch: refs/heads/hbase-12439
Commit: 938ee73e7cf27730adfed98e66be024899291f2d
Parents: 0206dc6
Author: Jurriaan Mous <ju...@jurmo.us>
Authored: Fri Jun 17 19:54:30 2016 +0200
Committer: Sean Busbey <bu...@apache.org>
Committed: Sat Aug 6 01:23:20 2016 -0500

----------------------------------------------------------------------
 dev-support/hbase-personality.sh | 75 +++++++++++++++++++++++++++--------
 1 file changed, 59 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/938ee73e/dev-support/hbase-personality.sh
----------------------------------------------------------------------
diff --git a/dev-support/hbase-personality.sh b/dev-support/hbase-personality.sh
index 3b6ebad..af397f0 100755
--- a/dev-support/hbase-personality.sh
+++ b/dev-support/hbase-personality.sh
@@ -36,8 +36,12 @@
 
 personality_plugins "all"
 
+## @description  Globals specific to this personality
+## @audience     private
+## @stability    evolving
 function personality_globals
 {
+  BUILDTOOL=maven
   #shellcheck disable=SC2034
   PROJECT_NAME=hbase
   #shellcheck disable=SC2034
@@ -58,6 +62,11 @@ function personality_globals
 
 }
 
+## @description  Queue up modules for this personality
+## @audience     private
+## @stability    evolving
+## @param        repostatus
+## @param        testtype
 function personality_modules
 {
   local repostatus=$1
@@ -71,13 +80,14 @@ function personality_modules
   extra="-DHBasePatchProcess"
 
   if [[ ${repostatus} == branch
-     && ${testtype} == mvninstall ]];then
-     personality_enqueue_module . ${extra}
-     return
+     && ${testtype} == mvninstall ]] ||
+     [[ "${BUILDMODE}" == full ]];then
+    personality_enqueue_module . ${extra}
+    return
   fi
 
   if [[ ${testtype} = findbugs ]]; then
-    for module in ${CHANGED_MODULES}; do
+    for module in "${CHANGED_MODULES[@]}"; do
       # skip findbugs on hbase-shell and hbase-it. hbase-it has nothing
       # in src/main/java where findbugs goes to look
       if [[ ${module} == hbase-shell ]]; then
@@ -130,7 +140,7 @@ function personality_modules
     fi
   fi
 
-  for module in ${CHANGED_MODULES}; do
+  for module in "${CHANGED_MODULES[@]}"; do
     # shellcheck disable=SC2086
     personality_enqueue_module ${module} ${extra}
   done
@@ -146,6 +156,10 @@ function personality_modules
 
 add_test_type hadoopcheck
 
+## @description  hadoopcheck file filter
+## @audience     private
+## @stability    evolving
+## @param        filename
 function hadoopcheck_filefilter
 {
   local filename=$1
@@ -155,6 +169,10 @@ function hadoopcheck_filefilter
   fi
 }
 
+## @description  hadoopcheck test
+## @audience     private
+## @stability    evolving
+## @param        repostatus
 function hadoopcheck_rebuild
 {
   local repostatus=$1
@@ -178,7 +196,7 @@ function hadoopcheck_rebuild
         -Dhadoop-two.version="${hadoopver}"
     count=$(${GREP} -c ERROR "${logfile}")
     if [[ ${count} -gt 0 ]]; then
-      add_vote_table -1 hadoopcheck "Patch causes ${count} errors with Hadoop v${hadoopver}."
+      add_vote_table -1 hadoopcheck "${BUILDMODEMSG} causes ${count} errors with Hadoop v${hadoopver}."
       ((result=result+1))
     fi
   done
@@ -193,9 +211,13 @@ function hadoopcheck_rebuild
 
 ######################################
 
-# TODO if we need th protoc check, we probably need to check building all the modules that rely on hbase-protocol
+# TODO if we need the protoc check, we probably need to check building all the modules that rely on hbase-protocol
 add_test_type hbaseprotoc
 
+## @description  hbaseprotoc file filter
+## @audience     private
+## @stability    evolving
+## @param        filename
 function hbaseprotoc_filefilter
 {
   local filename=$1
@@ -205,14 +227,19 @@ function hbaseprotoc_filefilter
   fi
 }
 
+## @description  hadoopcheck test
+## @audience     private
+## @stability    evolving
+## @param        repostatus
 function hbaseprotoc_rebuild
 {
-  local i=0
-  local fn
-  local module
-  local logfile
-  local count
-  local result
+  declare repostatus=$1
+  declare i=0
+  declare fn
+  declare module
+  declare logfile
+  declare count
+  declare result
 
   if [[ "${repostatus}" = branch ]]; then
     return 0
@@ -223,7 +250,7 @@ function hbaseprotoc_rebuild
     return 0
   fi
 
-  big_console_header "Patch HBase protoc plugin"
+  big_console_header "HBase protoc plugin: ${BUILDMODE}"
 
   start_clock
 
@@ -232,7 +259,7 @@ function hbaseprotoc_rebuild
   modules_workers patch hbaseprotoc compile -DskipTests -Pcompile-protobuf -X -DHBasePatchProcess
 
   # shellcheck disable=SC2153
-  until [[ $i -eq ${#MODULE[@]} ]]; do
+  until [[ $i -eq "${#MODULE[@]}" ]]; do
     if [[ ${MODULE_STATUS[${i}]} == -1 ]]; then
       ((result=result+1))
       ((i=i+1))
@@ -263,6 +290,10 @@ function hbaseprotoc_rebuild
 
 add_test_type hbaseanti
 
+## @description  hbaseanti file filter
+## @audience     private
+## @stability    evolving
+## @param        filename
 function hbaseanti_filefilter
 {
   local filename=$1
@@ -272,12 +303,20 @@ function hbaseanti_filefilter
   fi
 }
 
+## @description  hbaseanti patch file check
+## @audience     private
+## @stability    evolving
+## @param        filename
 function hbaseanti_patchfile
 {
   local patchfile=$1
   local warnings
   local result
 
+  if [[ "${BUILDMODE}" = full ]]; then
+    return 0
+  fi
+
   verify_needed_test hbaseanti
   if [[ $? == 0 ]]; then
     return 0
@@ -307,7 +346,11 @@ function hbaseanti_patchfile
   return 0
 }
 
-# Work around HBASE-15042
+
+## @description  hbase custom mvnsite file filter.  See HBASE-15042
+## @audience     private
+## @stability    evolving
+## @param        filename
 function mvnsite_filefilter
 {
   local filename=$1


[40/50] [abbrv] hbase git commit: REVERT of revert of "HBASE-16308 Contain protobuf references Gather up the pb references into a few locations only rather than have pb references distributed all about the code base." This is a revert of a revert; i.e. w

Posted by sy...@apache.org.
REVERT of revert of "HBASE-16308 Contain protobuf references Gather up the pb references into a few locations only rather than have pb references distributed all about the code base."
This is a revert of a revert; i.e. we are adding back the change only adding
back with fixes for the broken unit test; was a real issue on a test that
went in just at same time as this commit; I was getting a new nonce on each
retry rather than getting one for the mutation.

Other changes since revert are more hiding of RpcController. Use
accessor method rather than always pass in a RpcController

Walked back retrying operations that used to be single-shot (though
code comment said need a retry) because it opens a can of worms where
we retry stuff like bad column family when we shouldn't (needs
work adding in DoNotRetryIOEs)

Changed name of class from PayloadCarryingServerCallable to
CancellableRegionServerCallable.

Fix javadoc and findbugs warnings.

Fix case of not initializing the ScannerCallable RpcController.

Below is original commit message:

 Remove mention of ServiceException and other protobuf classes from all over the codebase.
 Purge TimeLimitedRpcController. Lets just have one override of RpcController.
        M hbase-client/src/main/java/org/apache/hadoop/hbase/client/AbstractRegionServerCallable.java
         Cleanup. Make it clear this is an odd class for async hbase intro.
        M hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
         Refactor of RegionServerCallable allows me clean up a bunch of
         boilerplate in here and remove protobuf references.
        M hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
          Purge protobuf references everywhere except a reference to a throw of a
          ServiceException in method checkHBaseAvailable. I deprecated it in favor
          of new available method (the SE is not actually needed)
        M hbase-client/src/main/java/org/apache/hadoop/hbase/client/PayloadCarryingServerCallable.java
         Move the RetryingTimeTracker instance in here from HTable.
         Allows me to contain tracker and remove a repeated code in HTable.
        M hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionServerCallable.java
         Clean up move set up of rpc in here rather than have it repeat in HTable.
         Allows me to remove protobuf references from a bunch of places.
    M hbase-client/src/main/java/org/apache/hadoop/hbase/client/FlushRegionCallable.java
     Make use of the push of boilerplate up into RegionServerCallable
    M hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java
    M hbase-client/src/main/java/org/apache/hadoop/hbase/client/PayloadCarryingServerCallable.java
    M hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionAdminServiceCallable.java
    M hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerWithReadReplicas.java
    M hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java
    M hbase-client/src/main/java/org/apache/hadoop/hbase/client/SecureBulkLoadClient.java
    M hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
     Move boilerplate up into superclass.
    M hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetryingTimeTracker.java
     Cleanup
    M hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/PayloadCarryingRpcController.java
    M hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
    M hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEditsReplaySink.java
    M hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java
     Factor in TimeLimitedRpcController. Just have one RpcController override.
    D hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/TimeLimitedRpcController.java
     Removed. Lets have one override of pb rpccontroller only.
    M hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
     (handleRemoteException) added
     (toText) added


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

Branch: refs/heads/hbase-12439
Commit: 45bb6180a3b8d915d8db88b8edf420cdbdcb4c21
Parents: 3c3457c
Author: stack <st...@apache.org>
Authored: Sun Aug 7 15:49:38 2016 -0700
Committer: stack <st...@apache.org>
Committed: Wed Aug 10 10:12:06 2016 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hbase/MetaTableAccessor.java  |    9 +-
 .../client/AbstractRegionServerCallable.java    |   23 +-
 .../hadoop/hbase/client/AsyncProcess.java       |   24 +-
 .../client/CancellableRegionServerCallable.java |   66 ++
 .../hadoop/hbase/client/ClientScanner.java      |    2 +-
 .../hbase/client/ClientSimpleScanner.java       |    3 +-
 .../hadoop/hbase/client/ClientSmallScanner.java |   46 +-
 .../hadoop/hbase/client/ConnectionCallable.java |   56 -
 .../hbase/client/ConnectionImplementation.java  |   40 +-
 .../hbase/client/FlushRegionCallable.java       |   26 +-
 .../apache/hadoop/hbase/client/HBaseAdmin.java  | 1085 ++++++++----------
 .../org/apache/hadoop/hbase/client/HTable.java  |  466 +++-----
 .../hadoop/hbase/client/MasterCallable.java     |   86 +-
 .../hbase/client/MasterKeepAliveConnection.java |    3 +-
 .../hbase/client/MultiServerCallable.java       |   36 +-
 .../client/NoncedRegionServerCallable.java      |  128 +++
 .../client/PayloadCarryingServerCallable.java   |   48 -
 .../client/RegionAdminServiceCallable.java      |   54 +-
 .../hbase/client/RegionServerCallable.java      |   96 +-
 .../hadoop/hbase/client/RetryingCallable.java   |    2 +-
 .../hbase/client/RetryingTimeTracker.java       |   12 +-
 .../hbase/client/ReversedScannerCallable.java   |    6 +-
 .../hbase/client/RpcRetryingCallable.java       |   65 ++
 .../hadoop/hbase/client/RpcRetryingCaller.java  |    5 +-
 .../hbase/client/RpcRetryingCallerFactory.java  |    1 +
 .../RpcRetryingCallerWithReadReplicas.java      |   30 +-
 .../hadoop/hbase/client/ScannerCallable.java    |  154 +--
 .../client/ScannerCallableWithReplicas.java     |    5 +-
 .../hbase/client/SecureBulkLoadClient.java      |   81 +-
 .../hbase/ipc/MasterCoprocessorRpcChannel.java  |    3 +-
 .../hbase/ipc/PayloadCarryingRpcController.java |  127 +-
 .../hbase/ipc/RegionCoprocessorRpcChannel.java  |   30 +-
 .../hadoop/hbase/ipc/RpcControllerFactory.java  |    3 +-
 .../hbase/ipc/TimeLimitedRpcController.java     |  142 ---
 .../hadoop/hbase/protobuf/ProtobufUtil.java     |   73 +-
 .../hadoop/hbase/client/TestAsyncProcess.java   |    6 +-
 .../hadoop/hbase/client/TestClientScanner.java  |    1 -
 .../apache/hadoop/hbase/HBaseIOException.java   |    3 +-
 .../apache/hadoop/hbase/util/ExceptionUtil.java |    2 +-
 .../org/apache/hadoop/hbase/ipc/RpcServer.java  |    2 +-
 .../hbase/mapreduce/LoadIncrementalHFiles.java  |   46 +-
 .../master/ExpiredMobFileCleanerChore.java      |    6 -
 .../hadoop/hbase/master/MasterRpcServices.java  |   17 +-
 .../hadoop/hbase/master/ServerManager.java      |    5 +-
 .../hadoop/hbase/master/TableStateManager.java  |    3 +-
 .../hadoop/hbase/mob/ExpiredMobFileCleaner.java |   12 +-
 .../hadoop/hbase/mob/mapreduce/Sweeper.java     |    6 +-
 .../hbase/regionserver/HRegionServer.java       |   40 +-
 .../hbase/regionserver/RSRpcServices.java       |   22 +-
 .../regionserver/wal/WALEditsReplaySink.java    |   43 +-
 .../RegionReplicaReplicationEndpoint.java       |   54 +-
 .../org/apache/hadoop/hbase/tool/Canary.java    |    7 +-
 .../org/apache/hadoop/hbase/util/Merge.java     |   13 +-
 .../org/apache/hadoop/hbase/TestNamespace.java  |    7 +-
 .../apache/hadoop/hbase/client/TestAdmin2.java  |   13 +-
 .../hadoop/hbase/client/TestClientTimeouts.java |    7 +-
 .../org/apache/hadoop/hbase/client/TestHCM.java |   40 +-
 .../hbase/client/TestReplicaWithCluster.java    |   52 +-
 .../hadoop/hbase/client/TestReplicasClient.java |    4 +-
 .../balancer/TestRegionLocationFinder.java      |    3 -
 .../hbase/mob/mapreduce/TestMobSweepMapper.java |    6 +-
 .../regionserver/TestHRegionServerBulkLoad.java |   22 +-
 .../TestHRegionServerBulkLoadWithOldClient.java |   13 +-
 ...gionServerBulkLoadWithOldSecureEndpoint.java |   26 +-
 .../TestScannerHeartbeatMessages.java           |    3 +-
 .../hbase/spark/SparkSQLPushDownFilter.java     |    4 +-
 66 files changed, 1745 insertions(+), 1779 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
index 1eaa753..2b50829 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
@@ -460,12 +460,9 @@ public class MetaTableAccessor {
    */
   public static List<HRegionInfo> getTableRegions(Connection connection,
       TableName tableName, final boolean excludeOfflinedSplitParents)
-      throws IOException {
-    List<Pair<HRegionInfo, ServerName>> result;
-
-    result = getTableRegionsAndLocations(connection, tableName,
-      excludeOfflinedSplitParents);
-
+  throws IOException {
+    List<Pair<HRegionInfo, ServerName>> result =
+        getTableRegionsAndLocations(connection, tableName, excludeOfflinedSplitParents);
     return getListOfHRegionInfos(result);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AbstractRegionServerCallable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AbstractRegionServerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AbstractRegionServerCallable.java
index 7279d81..5a1f5cc 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AbstractRegionServerCallable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AbstractRegionServerCallable.java
@@ -18,8 +18,7 @@
 package org.apache.hadoop.hbase.client;
 
 import java.io.IOException;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.ServerName;
@@ -29,26 +28,15 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.util.Bytes;
 
 /**
- * Implementations call a RegionServer.
- * Passed to a {@link RpcRetryingCaller} so we retry on fail.
- * TODO: this class is actually tied to one region, because most of the paths make use of
- *       the regioninfo part of location when building requests. The only reason it works for
- *       multi-region requests (e.g. batch) is that they happen to not use the region parts.
- *       This could be done cleaner (e.g. having a generic parameter and 2 derived classes,
- *       RegionCallable and actual RegionServerCallable with ServerName.
- * @param <T> the class that the ServerCallable handles
+ * Added by HBASE-15745 Refactor of RPC classes to better accept async changes.
+ * Temporary.
  */
 @InterfaceAudience.Private
 abstract class AbstractRegionServerCallable<T> implements RetryingCallable<T> {
-  // Public because used outside of this package over in ipc.
-  private static final Log LOG = LogFactory.getLog(AbstractRegionServerCallable.class);
-
   protected final Connection connection;
   protected final TableName tableName;
   protected final byte[] row;
-
   protected HRegionLocation location;
-
   protected final static int MIN_WAIT_DEAD_SERVER = 10000;
 
   /**
@@ -127,8 +115,7 @@ abstract class AbstractRegionServerCallable<T> implements RetryingCallable<T> {
   @Override
   public void prepare(final boolean reload) throws IOException {
     // check table state if this is a retry
-    if (reload &&
-        !tableName.equals(TableName.META_TABLE_NAME) &&
+    if (reload && !tableName.equals(TableName.META_TABLE_NAME) &&
         getConnection().isTableDisabled(tableName)) {
       throw new TableNotEnabledException(tableName.getNameAsString() + " is disabled.");
     }
@@ -148,4 +135,4 @@ abstract class AbstractRegionServerCallable<T> implements RetryingCallable<T> {
    * @throws IOException When client could not be created
    */
   abstract void setClientByServiceName(ServerName serviceName) throws IOException;
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
index 1383ca0..d699233 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
@@ -587,7 +587,7 @@ class AsyncProcess {
    */
   public <CResult> AsyncRequestFuture submitAll(ExecutorService pool, TableName tableName,
       List<? extends Row> rows, Batch.Callback<CResult> callback, Object[] results,
-      PayloadCarryingServerCallable callable, int curTimeout) {
+      CancellableRegionServerCallable callable, int curTimeout) {
     List<Action<Row>> actions = new ArrayList<Action<Row>>(rows.size());
 
     // The position will be used by the processBatch to match the object array returned.
@@ -739,11 +739,11 @@ class AsyncProcess {
       private final MultiAction<Row> multiAction;
       private final int numAttempt;
       private final ServerName server;
-      private final Set<PayloadCarryingServerCallable> callsInProgress;
+      private final Set<CancellableRegionServerCallable> callsInProgress;
 
       private SingleServerRequestRunnable(
           MultiAction<Row> multiAction, int numAttempt, ServerName server,
-          Set<PayloadCarryingServerCallable> callsInProgress) {
+          Set<CancellableRegionServerCallable> callsInProgress) {
         this.multiAction = multiAction;
         this.numAttempt = numAttempt;
         this.server = server;
@@ -753,7 +753,7 @@ class AsyncProcess {
       @Override
       public void run() {
         MultiResponse res;
-        PayloadCarryingServerCallable callable = currentCallable;
+        CancellableRegionServerCallable callable = currentCallable;
         try {
           // setup the callable based on the actions, if we don't have one already from the request
           if (callable == null) {
@@ -802,7 +802,7 @@ class AsyncProcess {
     private final BatchErrors errors;
     private final ConnectionImplementation.ServerErrorTracker errorsByServer;
     private final ExecutorService pool;
-    private final Set<PayloadCarryingServerCallable> callsInProgress;
+    private final Set<CancellableRegionServerCallable> callsInProgress;
 
 
     private final TableName tableName;
@@ -829,12 +829,12 @@ class AsyncProcess {
     private final int[] replicaGetIndices;
     private final boolean hasAnyReplicaGets;
     private final long nonceGroup;
-    private PayloadCarryingServerCallable currentCallable;
+    private CancellableRegionServerCallable currentCallable;
     private int currentCallTotalTimeout;
 
     public AsyncRequestFutureImpl(TableName tableName, List<Action<Row>> actions, long nonceGroup,
         ExecutorService pool, boolean needResults, Object[] results,
-        Batch.Callback<CResult> callback, PayloadCarryingServerCallable callable, int timeout) {
+        Batch.Callback<CResult> callback, CancellableRegionServerCallable callable, int timeout) {
       this.pool = pool;
       this.callback = callback;
       this.nonceGroup = nonceGroup;
@@ -899,7 +899,7 @@ class AsyncProcess {
       }
       this.callsInProgress = !hasAnyReplicaGets ? null :
           Collections.newSetFromMap(
-              new ConcurrentHashMap<PayloadCarryingServerCallable, Boolean>());
+              new ConcurrentHashMap<CancellableRegionServerCallable, Boolean>());
 
       this.errorsByServer = createServerErrorTracker();
       this.errors = (globalErrors != null) ? globalErrors : new BatchErrors();
@@ -907,7 +907,7 @@ class AsyncProcess {
       this.currentCallTotalTimeout = timeout;
     }
 
-    public Set<PayloadCarryingServerCallable> getCallsInProgress() {
+    public Set<CancellableRegionServerCallable> getCallsInProgress() {
       return callsInProgress;
     }
 
@@ -1662,7 +1662,7 @@ class AsyncProcess {
         throw new InterruptedIOException(iex.getMessage());
       } finally {
         if (callsInProgress != null) {
-          for (PayloadCarryingServerCallable clb : callsInProgress) {
+          for (CancellableRegionServerCallable clb : callsInProgress) {
             clb.cancel();
           }
         }
@@ -1743,7 +1743,7 @@ class AsyncProcess {
   protected <CResult> AsyncRequestFutureImpl<CResult> createAsyncRequestFuture(
       TableName tableName, List<Action<Row>> actions, long nonceGroup, ExecutorService pool,
       Batch.Callback<CResult> callback, Object[] results, boolean needResults,
-      PayloadCarryingServerCallable callable, int curTimeout) {
+      CancellableRegionServerCallable callable, int curTimeout) {
     return new AsyncRequestFutureImpl<CResult>(
         tableName, actions, nonceGroup, getPool(pool), needResults,
         results, callback, callable, curTimeout);
@@ -1771,7 +1771,7 @@ class AsyncProcess {
    * Create a caller. Isolated to be easily overridden in the tests.
    */
   @VisibleForTesting
-  protected RpcRetryingCaller<MultiResponse> createCaller(PayloadCarryingServerCallable callable) {
+  protected RpcRetryingCaller<MultiResponse> createCaller(CancellableRegionServerCallable callable) {
     return rpcCallerFactory.<MultiResponse> newCaller();
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-client/src/main/java/org/apache/hadoop/hbase/client/CancellableRegionServerCallable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/CancellableRegionServerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/CancellableRegionServerCallable.java
new file mode 100644
index 0000000..0a6e10f
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/CancellableRegionServerCallable.java
@@ -0,0 +1,66 @@
+/*
+ * 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 regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
+
+/**
+ * This class is used to unify HTable calls with AsyncProcess Framework. HTable can use
+ * AsyncProcess directly though this class. Also adds global timeout tracking on top of
+ * RegionServerCallable and implements Cancellable.
+ */
+@InterfaceAudience.Private
+abstract class CancellableRegionServerCallable<T> extends RegionServerCallable<T> implements
+Cancellable {
+  private final RetryingTimeTracker tracker = new RetryingTimeTracker();
+
+  CancellableRegionServerCallable(Connection connection, TableName tableName, byte[] row,
+      RpcControllerFactory rpcControllerFactory) {
+    super(connection, rpcControllerFactory, tableName, row);
+  }
+
+  /* Override so can mess with the callTimeout.
+   * (non-Javadoc)
+   * @see org.apache.hadoop.hbase.client.RegionServerCallable#rpcCall(int)
+   */
+  @Override
+  public T call(int callTimeout) throws IOException {
+    // It is expected (it seems) that tracker.start can be called multiple times (on each trip
+    // through the call when retrying). Also, we can call start and no need of a stop.
+    this.tracker.start();
+    int remainingTime = tracker.getRemainingTime(callTimeout);
+    if (remainingTime == 0) {
+      throw new DoNotRetryIOException("Timeout for mutate row");
+    }
+    return super.call(remainingTime);
+  }
+
+  @Override
+  public void cancel() {
+    getRpcController().startCancel();
+  }
+
+  @Override
+  public boolean isCancelled() {
+    return getRpcController().isCanceled();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java
index cb4c714..3e676c7 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java
@@ -847,4 +847,4 @@ public abstract class ClientScanner extends AbstractClientScanner {
     Cell[] list = Arrays.copyOfRange(result.rawCells(), index, result.rawCells().length);
     return Result.create(list, result.getExists(), result.isStale(), result.isPartial());
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSimpleScanner.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSimpleScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSimpleScanner.java
index f886971..ecf083b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSimpleScanner.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSimpleScanner.java
@@ -32,7 +32,6 @@ import java.util.concurrent.ExecutorService;
  */
 @InterfaceAudience.Private
 public class ClientSimpleScanner extends ClientScanner {
-
   public ClientSimpleScanner(Configuration configuration, Scan scan, TableName name,
       ClusterConnection connection, RpcRetryingCallerFactory rpcCallerFactory,
       RpcControllerFactory rpcControllerFactory, ExecutorService pool,
@@ -50,4 +49,4 @@ public class ClientSimpleScanner extends ClientScanner {
   public Result next() throws IOException {
     return nextWithSyncCache();
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallScanner.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallScanner.java
index f9bdd55..f13f3f4 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallScanner.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallScanner.java
@@ -18,8 +18,10 @@
  */
 package org.apache.hadoop.hbase.client;
 
-import com.google.common.annotations.VisibleForTesting;
-import com.google.protobuf.ServiceException;
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.util.concurrent.ExecutorService;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -30,16 +32,13 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
-import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.protobuf.ResponseConverter;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse;
 import org.apache.hadoop.hbase.util.Bytes;
 
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.util.concurrent.ExecutorService;
+import com.google.common.annotations.VisibleForTesting;
 
 /**
  * Client scanner for small scan. Generally, only one RPC is called to fetch the
@@ -185,7 +184,7 @@ public class ClientSmallScanner extends ClientSimpleScanner {
     }
 
     @Override
-    public Result[] call(int timeout) throws IOException {
+    protected Result[] rpcCall() throws Exception {
       if (this.closed) return null;
       if (Thread.interrupted()) {
         throw new InterruptedIOException();
@@ -193,31 +192,23 @@ public class ClientSmallScanner extends ClientSimpleScanner {
       ScanRequest request = RequestConverter.buildScanRequest(getLocation()
           .getRegionInfo().getRegionName(), getScan(), getCaching(), true);
       ScanResponse response = null;
-      controller = controllerFactory.newController();
-      try {
-        controller.setPriority(getTableName());
-        controller.setCallTimeout(timeout);
-        response = getStub().scan(controller, request);
-        Result[] results = ResponseConverter.getResults(controller.cellScanner(),
-            response);
-        if (response.hasMoreResultsInRegion()) {
-          setHasMoreResultsContext(true);
-          setServerHasMoreResults(response.getMoreResultsInRegion());
-        } else {
-          setHasMoreResultsContext(false);
-        }
-        // We need to update result metrics since we are overriding call()
-        updateResultsMetrics(results);
-        return results;
-      } catch (ServiceException se) {
-        throw ProtobufUtil.getRemoteException(se);
+      response = getStub().scan(getRpcController(), request);
+      Result[] results = ResponseConverter.getResults(getRpcControllerCellScanner(), response);
+      if (response.hasMoreResultsInRegion()) {
+        setHasMoreResultsContext(true);
+        setServerHasMoreResults(response.getMoreResultsInRegion());
+      } else {
+        setHasMoreResultsContext(false);
       }
+      // We need to update result metrics since we are overriding call()
+      updateResultsMetrics(results);
+      return results;
     }
 
     @Override
     public ScannerCallable getScannerCallableForReplica(int id) {
       return new SmallScannerCallable((ClusterConnection)connection, tableName, getScan(),
-          scanMetrics, controllerFactory, getCaching(), id);
+          scanMetrics, rpcControllerFactory, getCaching(), id);
     }
   }
 
@@ -311,6 +302,5 @@ public class ClientSmallScanner extends ClientSimpleScanner {
               scannerTimeout, cacheNum, conf, caller);
       return scannerCallableWithReplicas;
     }
-
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionCallable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionCallable.java
deleted file mode 100644
index 3f44927..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionCallable.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/**
- * 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
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.client;
-
-import java.io.Closeable;
-import java.io.IOException;
-
-/**
- * A RetryingCallable for generic connection operations.
- * @param <V> return type
- */
-abstract class ConnectionCallable<V> implements RetryingCallable<V>, Closeable {
-  protected Connection connection;
-
-  public ConnectionCallable(final Connection connection) {
-    this.connection = connection;
-  }
-
-  @Override
-  public void prepare(boolean reload) throws IOException {
-  }
-
-  @Override
-  public void close() throws IOException {
-  }
-
-  @Override
-  public void throwable(Throwable t, boolean retrying) {
-  }
-
-  @Override
-  public String getExceptionMessageAdditionalDetail() {
-    return "";
-  }
-
-  @Override
-  public long sleep(long pause, int tries) {
-    return ConnectionUtils.getPauseTime(pause, tries);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
index 37c62c5..38178b4 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
@@ -20,11 +20,6 @@ package org.apache.hadoop.hbase.client;
 
 import static org.apache.hadoop.hbase.client.MetricsConnection.CLIENT_SIDE_METRICS_ENABLED_KEY;
 
-import com.google.common.annotations.VisibleForTesting;
-import com.google.protobuf.BlockingRpcChannel;
-import com.google.protobuf.RpcController;
-import com.google.protobuf.ServiceException;
-
 import java.io.Closeable;
 import java.io.IOException;
 import java.io.InterruptedIOException;
@@ -68,6 +63,7 @@ import org.apache.hadoop.hbase.exceptions.RegionMovedException;
 import org.apache.hadoop.hbase.ipc.RpcClient;
 import org.apache.hadoop.hbase.ipc.RpcClientFactory;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
@@ -95,6 +91,11 @@ import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.zookeeper.KeeperException;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.BlockingRpcChannel;
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+
 /**
  * Main implementation of {@link Connection} and {@link ClusterConnection} interfaces.
  * Encapsulates connection to zookeeper and regionservers.
@@ -934,9 +935,13 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
       this.stub = null;
     }
 
-    boolean isMasterRunning() throws ServiceException {
-      MasterProtos.IsMasterRunningResponse response =
-        this.stub.isMasterRunning(null, RequestConverter.buildIsMasterRunningRequest());
+    boolean isMasterRunning() throws IOException {
+      MasterProtos.IsMasterRunningResponse response = null;
+      try {
+        response = this.stub.isMasterRunning(null, RequestConverter.buildIsMasterRunningRequest());
+      } catch (Exception e) {
+        throw ProtobufUtil.handleRemoteException(e);
+      }
       return response != null? response.getIsMasterRunning(): false;
     }
   }
@@ -1059,14 +1064,14 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
     /**
      * Once setup, check it works by doing isMasterRunning check.
      */
-    protected abstract void isMasterRunning() throws ServiceException;
+    protected abstract void isMasterRunning() throws IOException;
 
     /**
      * Create a stub. Try once only.  It is not typed because there is no common type to
      * protobuf services nor their interfaces.  Let the caller do appropriate casting.
      * @return A stub for master services.
      */
-    private Object makeStubNoRetries() throws IOException, KeeperException, ServiceException {
+    private Object makeStubNoRetries() throws IOException, KeeperException {
       ZooKeeperKeepAliveConnection zkw;
       try {
         zkw = getKeepAliveZooKeeperWatcher();
@@ -1106,7 +1111,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
     }
 
     /**
-     * Create a stub against the master.  Retry if necessary.
+     * Create a stub against the master. Retry if necessary.
      * @return A stub to do <code>intf</code> against the master
      * @throws org.apache.hadoop.hbase.MasterNotRunningException if master is not running
      */
@@ -1122,10 +1127,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
             exceptionCaught = e;
           } catch (KeeperException e) {
             exceptionCaught = e;
-          } catch (ServiceException e) {
-            exceptionCaught = e;
           }
-
           throw new MasterNotRunningException(exceptionCaught);
         } else {
           throw new DoNotRetryIOException("Connection was closed while trying to get master");
@@ -1156,8 +1158,12 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
     }
 
     @Override
-    protected void isMasterRunning() throws ServiceException {
-      this.stub.isMasterRunning(null, RequestConverter.buildIsMasterRunningRequest());
+    protected void isMasterRunning() throws IOException {
+      try {
+        this.stub.isMasterRunning(null, RequestConverter.buildIsMasterRunningRequest());
+      } catch (Exception e) {
+        throw ProtobufUtil.handleRemoteException(e);
+      }
     }
   }
 
@@ -1702,7 +1708,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
       //  java.net.ConnectException but they're not declared. So we catch it...
       LOG.info("Master connection is not running anymore", e.getUndeclaredThrowable());
       return false;
-    } catch (ServiceException se) {
+    } catch (IOException se) {
       LOG.warn("Checking master connection", se);
       return false;
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FlushRegionCallable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FlushRegionCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FlushRegionCallable.java
index 73bdb74..c7bf804 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FlushRegionCallable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FlushRegionCallable.java
@@ -27,23 +27,18 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
-import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.FlushRegionRequest;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.FlushRegionResponse;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 
-import com.google.protobuf.ServiceException;
-
 /**
  * A Callable for flushRegion() RPC.
  */
 @InterfaceAudience.Private
 public class FlushRegionCallable extends RegionAdminServiceCallable<FlushRegionResponse> {
-
   private static final Log LOG = LogFactory.getLog(FlushRegionCallable.class);
-
   private final byte[] regionName;
   private final boolean writeFlushWalMarker;
   private boolean reload;
@@ -64,18 +59,14 @@ public class FlushRegionCallable extends RegionAdminServiceCallable<FlushRegionR
   }
 
   @Override
-  public FlushRegionResponse call(int callTimeout) throws Exception {
-    return flushRegion();
-  }
-
-  @Override
   public void prepare(boolean reload) throws IOException {
     super.prepare(reload);
     this.reload = reload;
   }
 
-  private FlushRegionResponse flushRegion() throws IOException {
-    // check whether we should still do the flush to this region. If the regions are changed due
+  @Override
+  protected FlushRegionResponse call(PayloadCarryingRpcController controller) throws Exception {
+    // Check whether we should still do the flush to this region. If the regions are changed due
     // to splits or merges, etc return success
     if (!Bytes.equals(location.getRegionInfo().getRegionName(), regionName)) {
       if (!reload) {
@@ -93,13 +84,6 @@ public class FlushRegionCallable extends RegionAdminServiceCallable<FlushRegionR
 
     FlushRegionRequest request =
         RequestConverter.buildFlushRegionRequest(regionName, writeFlushWalMarker);
-
-    try {
-      PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-      controller.setPriority(tableName);
-      return stub.flushRegion(controller, request);
-    } catch (ServiceException se) {
-      throw ProtobufUtil.getRemoteException(se);
-    }
+    return stub.flushRegion(controller, request);
   }
-}
+}
\ No newline at end of file


[12/50] [abbrv] hbase git commit: HBASE-8386 deprecate TableMapReduce.addDependencyJars(Configuration, class ...)

Posted by sy...@apache.org.
HBASE-8386 deprecate TableMapReduce.addDependencyJars(Configuration, class<?> ...)

Signed-off-by: Matteo Bertozzi <ma...@cloudera.com>


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

Branch: refs/heads/hbase-12439
Commit: 431c8c9ad03336b5ab844d97adc47b8a73d7aa00
Parents: c59f764
Author: Sean Busbey <bu...@apache.org>
Authored: Tue Jul 19 07:21:26 2016 -0500
Committer: Sean Busbey <bu...@apache.org>
Committed: Fri Aug 5 10:35:02 2016 -0500

----------------------------------------------------------------------
 .../test/IntegrationTestBigLinkedList.java      |  6 ++--
 .../test/IntegrationTestLoadAndVerify.java      |  4 +--
 ...tionTestWithCellVisibilityLoadAndVerify.java |  2 +-
 .../hadoop/hbase/mapred/TableMapReduceUtil.java |  2 +-
 .../apache/hadoop/hbase/mapreduce/Import.java   |  6 ++--
 .../hadoop/hbase/mapreduce/ImportTsv.java       |  2 +-
 .../hbase/mapreduce/TableMapReduceUtil.java     | 30 ++++++++++++++++++--
 .../hadoop/hbase/mapreduce/WALPlayer.java       |  4 +--
 .../hadoop/hbase/PerformanceEvaluation.java     |  2 +-
 .../mapred/TestTableSnapshotInputFormat.java    |  2 +-
 .../mapreduce/TestTableSnapshotInputFormat.java |  2 +-
 src/main/asciidoc/_chapters/mapreduce.adoc      |  2 +-
 12 files changed, 45 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/431c8c9a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
index 6c54dca..135bea7 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
@@ -785,7 +785,8 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase {
 
       job.getConfiguration().setBoolean("mapreduce.map.speculative", false);
       TableMapReduceUtil.addDependencyJars(job);
-      TableMapReduceUtil.addDependencyJars(job.getConfiguration(), AbstractHBaseTool.class);
+      TableMapReduceUtil.addDependencyJarsForClasses(job.getConfiguration(),
+                                                     AbstractHBaseTool.class);
       TableMapReduceUtil.initCredentials(job);
 
       boolean success = jobCompletion(job);
@@ -1296,7 +1297,8 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase {
 
       TableMapReduceUtil.initTableMapperJob(getTableName(getConf()).getName(), scan,
           VerifyMapper.class, BytesWritable.class, BytesWritable.class, job);
-      TableMapReduceUtil.addDependencyJars(job.getConfiguration(), AbstractHBaseTool.class);
+      TableMapReduceUtil.addDependencyJarsForClasses(job.getConfiguration(),
+                                                     AbstractHBaseTool.class);
 
       job.getConfiguration().setBoolean("mapreduce.map.speculative", false);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/431c8c9a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java
index e279dfb..d12383d 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java
@@ -348,7 +348,7 @@ public void cleanUpCluster() throws Exception {
 
     TableMapReduceUtil.addDependencyJars(job);
 
-    TableMapReduceUtil.addDependencyJars(job.getConfiguration(), AbstractHBaseTool.class);
+    TableMapReduceUtil.addDependencyJarsForClasses(job.getConfiguration(), AbstractHBaseTool.class);
     TableMapReduceUtil.initCredentials(job);
     assertTrue(job.waitForCompletion(true));
     return job;
@@ -372,7 +372,7 @@ public void cleanUpCluster() throws Exception {
     TableMapReduceUtil.initTableMapperJob(
         htd.getTableName().getNameAsString(), scan, VerifyMapper.class,
         BytesWritable.class, BytesWritable.class, job);
-    TableMapReduceUtil.addDependencyJars(job.getConfiguration(), AbstractHBaseTool.class);
+    TableMapReduceUtil.addDependencyJarsForClasses(job.getConfiguration(), AbstractHBaseTool.class);
     int scannerCaching = conf.getInt("verify.scannercaching", SCANNER_CACHING);
     TableMapReduceUtil.setScannerCaching(job, scannerCaching);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/431c8c9a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestWithCellVisibilityLoadAndVerify.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestWithCellVisibilityLoadAndVerify.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestWithCellVisibilityLoadAndVerify.java
index 52a705b..abaa481 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestWithCellVisibilityLoadAndVerify.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestWithCellVisibilityLoadAndVerify.java
@@ -331,7 +331,7 @@ public class IntegrationTestWithCellVisibilityLoadAndVerify extends IntegrationT
     scan.setAuthorizations(new Authorizations(auths));
     TableMapReduceUtil.initTableMapperJob(htd.getTableName().getNameAsString(), scan,
         VerifyMapper.class, NullWritable.class, NullWritable.class, job);
-    TableMapReduceUtil.addDependencyJars(job.getConfiguration(), AbstractHBaseTool.class);
+    TableMapReduceUtil.addDependencyJarsForClasses(job.getConfiguration(), AbstractHBaseTool.class);
     int scannerCaching = conf.getInt("verify.scannercaching", SCANNER_CACHING);
     TableMapReduceUtil.setScannerCaching(job, scannerCaching);
     job.setNumReduceTasks(0);

http://git-wip-us.apache.org/repos/asf/hbase/blob/431c8c9a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableMapReduceUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableMapReduceUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableMapReduceUtil.java
index 84a279d..73340c4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableMapReduceUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableMapReduceUtil.java
@@ -362,7 +362,7 @@ public class TableMapReduceUtil {
    */
   public static void addDependencyJars(JobConf job) throws IOException {
     org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil.addHBaseDependencyJars(job);
-    org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil.addDependencyJars(
+    org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil.addDependencyJarsForClasses(
       job,
       // when making changes here, consider also mapreduce.TableMapReduceUtil
       // pull job classes

http://git-wip-us.apache.org/repos/asf/hbase/blob/431c8c9a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java
index c7d5912..e2693b9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java
@@ -616,7 +616,7 @@ public class Import extends Configured implements Tool {
     try {
       Class<? extends Filter> filter = conf.getClass(FILTER_CLASS_CONF_KEY, null, Filter.class);
       if (filter != null) {
-        TableMapReduceUtil.addDependencyJars(conf, filter);
+        TableMapReduceUtil.addDependencyJarsForClasses(conf, filter);
       }
     } catch (Exception e) {
       throw new IOException(e);
@@ -643,7 +643,7 @@ public class Import extends Configured implements Tool {
         fs.deleteOnExit(partitionsPath);
         job.setPartitionerClass(KeyValueWritableComparablePartitioner.class);
         job.setNumReduceTasks(regionLocator.getStartKeys().length);
-        TableMapReduceUtil.addDependencyJars(job.getConfiguration(),
+        TableMapReduceUtil.addDependencyJarsForClasses(job.getConfiguration(),
             com.google.common.base.Preconditions.class);
       }
     } else if (hfileOutPath != null) {
@@ -658,7 +658,7 @@ public class Import extends Configured implements Tool {
         job.setMapOutputKeyClass(ImmutableBytesWritable.class);
         job.setMapOutputValueClass(KeyValue.class);
         HFileOutputFormat2.configureIncrementalLoad(job, table.getTableDescriptor(), regionLocator);
-        TableMapReduceUtil.addDependencyJars(job.getConfiguration(),
+        TableMapReduceUtil.addDependencyJarsForClasses(job.getConfiguration(),
             com.google.common.base.Preconditions.class);
       }
     } else {

http://git-wip-us.apache.org/repos/asf/hbase/blob/431c8c9a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java
index e778d99..39085df 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java
@@ -587,7 +587,7 @@ public class ImportTsv extends Configured implements Tool {
               KeyValueSerialization.class.getName());
         }
         TableMapReduceUtil.addDependencyJars(job);
-        TableMapReduceUtil.addDependencyJars(job.getConfiguration(),
+        TableMapReduceUtil.addDependencyJarsForClasses(job.getConfiguration(),
             com.google.common.base.Function.class /* Guava used by TsvParser */);
       }
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/431c8c9a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java
index 37e4e44..8d8a784 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java
@@ -339,7 +339,7 @@ public class TableMapReduceUtil {
 
     if (addDependencyJars) {
       addDependencyJars(job);
-      addDependencyJars(job.getConfiguration(), MetricRegistry.class);
+      addDependencyJarsForClasses(job.getConfiguration(), MetricRegistry.class);
     }
 
     resetCacheConfig(job.getConfiguration());
@@ -787,7 +787,7 @@ public class TableMapReduceUtil {
           "  Continuing without it.");
     }
 
-    addDependencyJars(conf,
+    addDependencyJarsForClasses(conf,
       // explicitly pull a class from each module
       org.apache.hadoop.hbase.HConstants.class,                      // hbase-common
       org.apache.hadoop.hbase.protobuf.generated.ClientProtos.class, // hbase-protocol
@@ -835,7 +835,7 @@ public class TableMapReduceUtil {
   public static void addDependencyJars(Job job) throws IOException {
     addHBaseDependencyJars(job.getConfiguration());
     try {
-      addDependencyJars(job.getConfiguration(),
+      addDependencyJarsForClasses(job.getConfiguration(),
           // when making changes here, consider also mapred.TableMapReduceUtil
           // pull job classes
           job.getMapOutputKeyClass(),
@@ -855,9 +855,33 @@ public class TableMapReduceUtil {
    * Add the jars containing the given classes to the job's configuration
    * such that JobClient will ship them to the cluster and add them to
    * the DistributedCache.
+   * @deprecated rely on {@link #addDependencyJars(Job)} instead.
    */
+  @Deprecated
   public static void addDependencyJars(Configuration conf,
       Class<?>... classes) throws IOException {
+    LOG.warn("The addDependencyJars(Configuration, Class<?>...) method has been deprecated since it"
+             + " is easy to use incorrectly. Most users should rely on addDependencyJars(Job) " +
+             "instead. See HBASE-8386 for more details.");
+    addDependencyJarsForClasses(conf, classes);
+  }
+
+  /**
+   * Add the jars containing the given classes to the job's configuration
+   * such that JobClient will ship them to the cluster and add them to
+   * the DistributedCache.
+   *
+   * N.B. that this method at most adds one jar per class given. If there is more than one
+   * jar available containing a class with the same name as a given class, we don't define
+   * which of those jars might be chosen.
+   *
+   * @param conf The Hadoop Configuration to modify
+   * @param classes will add just those dependencies needed to find the given classes
+   * @throws IOException if an underlying library call fails.
+   */
+  @InterfaceAudience.Private
+  public static void addDependencyJarsForClasses(Configuration conf,
+      Class<?>... classes) throws IOException {
 
     FileSystem localFs = FileSystem.getLocal(conf);
     Set<String> jars = new HashSet<String>();

http://git-wip-us.apache.org/repos/asf/hbase/blob/431c8c9a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java
index c6fefb2..452714b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java
@@ -282,7 +282,7 @@ public class WALPlayer extends Configured implements Tool {
           RegionLocator regionLocator = conn.getRegionLocator(tableName)) {
         HFileOutputFormat2.configureIncrementalLoad(job, table.getTableDescriptor(), regionLocator);
       }
-      TableMapReduceUtil.addDependencyJars(job.getConfiguration(),
+      TableMapReduceUtil.addDependencyJarsForClasses(job.getConfiguration(),
           com.google.common.base.Preconditions.class);
     } else {
       // output to live cluster
@@ -295,7 +295,7 @@ public class WALPlayer extends Configured implements Tool {
     }
     String codecCls = WALCellCodec.getWALCellCodecClass(conf);
     try {
-      TableMapReduceUtil.addDependencyJars(job.getConfiguration(), Class.forName(codecCls));
+      TableMapReduceUtil.addDependencyJarsForClasses(job.getConfiguration(), Class.forName(codecCls));
     } catch (Exception e) {
       throw new IOException("Cannot determine wal codec class " + codecCls, e);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/431c8c9a/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
index 0528975..dc8a783 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
@@ -499,7 +499,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
     TextOutputFormat.setOutputPath(job, new Path(inputDir.getParent(), "outputs"));
 
     TableMapReduceUtil.addDependencyJars(job);
-    TableMapReduceUtil.addDependencyJars(job.getConfiguration(),
+    TableMapReduceUtil.addDependencyJarsForClasses(job.getConfiguration(),
       Histogram.class,     // yammer metrics
       ObjectMapper.class); // jackson-mapper-asl
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/431c8c9a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableSnapshotInputFormat.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableSnapshotInputFormat.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableSnapshotInputFormat.java
index c85cead..c4fdea4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableSnapshotInputFormat.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableSnapshotInputFormat.java
@@ -243,7 +243,7 @@ public class TestTableSnapshotInputFormat extends TableSnapshotInputFormatTestBa
       JobConf jobConf = new JobConf(util.getConfiguration());
 
       jobConf.setJarByClass(util.getClass());
-      org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil.addDependencyJars(jobConf,
+      org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil.addDependencyJarsForClasses(jobConf,
         TestTableSnapshotInputFormat.class);
 
       TableMapReduceUtil.initTableSnapshotMapJob(snapshotName, COLUMNS,

http://git-wip-us.apache.org/repos/asf/hbase/blob/431c8c9a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableSnapshotInputFormat.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableSnapshotInputFormat.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableSnapshotInputFormat.java
index 7c10def..5e8c8cd 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableSnapshotInputFormat.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableSnapshotInputFormat.java
@@ -281,7 +281,7 @@ public class TestTableSnapshotInputFormat extends TableSnapshotInputFormatTestBa
       Scan scan = new Scan(startRow, endRow); // limit the scan
 
       job.setJarByClass(util.getClass());
-      TableMapReduceUtil.addDependencyJars(job.getConfiguration(),
+      TableMapReduceUtil.addDependencyJarsForClasses(job.getConfiguration(),
         TestTableSnapshotInputFormat.class);
 
       TableMapReduceUtil.initTableSnapshotMapperJob(snapshotName,

http://git-wip-us.apache.org/repos/asf/hbase/blob/431c8c9a/src/main/asciidoc/_chapters/mapreduce.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/mapreduce.adoc b/src/main/asciidoc/_chapters/mapreduce.adoc
index 75718fd..dfa843a 100644
--- a/src/main/asciidoc/_chapters/mapreduce.adoc
+++ b/src/main/asciidoc/_chapters/mapreduce.adoc
@@ -73,7 +73,7 @@ This example assumes you use a BASH-compatible shell.
 $ HADOOP_CLASSPATH=`${HBASE_HOME}/bin/hbase classpath` ${HADOOP_HOME}/bin/hadoop jar ${HBASE_HOME}/lib/hbase-server-VERSION.jar rowcounter usertable
 ----
 
-When the command runs, internally, the HBase JAR finds the dependencies it needs for ZooKeeper, Guava, and its other dependencies on the passed `HADOOP_CLASSPATH` and adds the JARs to the MapReduce job configuration.
+When the command runs, internally, the HBase JAR finds the dependencies it needs and adds them to the MapReduce job configuration.
 See the source at `TableMapReduceUtil#addDependencyJars(org.apache.hadoop.mapreduce.Job)` for how this is done.
 
 The command `hbase mapredcp` can also help you dump the CLASSPATH entries required by MapReduce, which are the same jars `TableMapReduceUtil#addDependencyJars` would add.


[05/50] [abbrv] hbase git commit: HBASE-16354 Clean up defunct github PRs

Posted by sy...@apache.org.
HBASE-16354 Clean up defunct github PRs

* closes #34 mistaken branch-1.3 merge
* closes #32 no response from contributor in a month
* closes #31 no response from contributor in a month
* closes #30 no response from contributor in 2.5 months
* closes #29 fixed in HBASE-15863
* closes #25 no response from contributor in 4.5 months and obviated by HBASE-16340
* closes #24 no response from contributor in 4.5 months
* closes #22 jira HBASE-14442 closed as stale
* closes #16 fixed in HBASE-15574
* closes #13 no response from contributor in 4.5 months
* closes #11 no response from contributor in 4.5 months and obviated by HBASE-15265
* closes #4 no response from contributor in 4.5 months

Signed-off-by: Dima Spivak <ds...@cloudera.com>
Signed-off-by: Esteban Gutierrez <es...@apache.org>


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

Branch: refs/heads/hbase-12439
Commit: c9f84e8137c451c797101146c9049f6dd440c662
Parents: 975f0dd
Author: Sean Busbey <bu...@apache.org>
Authored: Thu Aug 4 02:02:56 2016 -0500
Committer: Sean Busbey <bu...@apache.org>
Committed: Thu Aug 4 16:25:57 2016 -0500

----------------------------------------------------------------------

----------------------------------------------------------------------



[43/50] [abbrv] hbase git commit: HBASE-16363 Correct javadoc for qualifier length and value (Xiang Li)

Posted by sy...@apache.org.
HBASE-16363 Correct javadoc for qualifier length and value (Xiang Li)


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

Branch: refs/heads/hbase-12439
Commit: 7e3251051313d6ef8fd173c7b530c1822667410e
Parents: 8895667
Author: stack <st...@apache.org>
Authored: Wed Aug 10 17:53:13 2016 -0700
Committer: stack <st...@apache.org>
Committed: Wed Aug 10 17:53:13 2016 -0700

----------------------------------------------------------------------
 hbase-common/src/main/java/org/apache/hadoop/hbase/Cell.java | 5 ++---
 1 file changed, 2 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/7e325105/hbase-common/src/main/java/org/apache/hadoop/hbase/Cell.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/Cell.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/Cell.java
index 5673ec9..626f041 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/Cell.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/Cell.java
@@ -106,8 +106,7 @@ public interface Cell {
   //3) Qualifier
 
   /**
-   * Contiguous raw bytes that may start at any index in the containing array. Max length is
-   * Short.MAX_VALUE which is 32,767 bytes.
+   * Contiguous raw bytes that may start at any index in the containing array.
    * @return The array containing the qualifier bytes.
    */
   byte[] getQualifierArray();
@@ -155,7 +154,7 @@ public interface Cell {
 
   /**
    * Contiguous raw bytes that may start at any index in the containing array. Max length is
-   * Integer.MAX_VALUE which is 2,147,483,648 bytes.
+   * Integer.MAX_VALUE which is 2,147,483,647 bytes.
    * @return The array containing the value bytes.
    */
   byte[] getValueArray();


[36/50] [abbrv] hbase git commit: REVERT of revert of "HBASE-16308 Contain protobuf references Gather up the pb references into a few locations only rather than have pb references distributed all about the code base." This is a revert of a revert; i.e. w

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionLocationFinder.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionLocationFinder.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionLocationFinder.java
index 0cccce1..daa8942 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionLocationFinder.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionLocationFinder.java
@@ -23,8 +23,6 @@ import static org.junit.Assert.assertTrue;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HDFSBlocksDistribution;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
@@ -43,7 +41,6 @@ import org.junit.experimental.categories.Category;
 
 @Category({MasterTests.class, SmallTests.class})
 public class TestRegionLocationFinder {
-  private static final Log LOG = LogFactory.getLog(TestRegionLocationFinder.class);
   private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
   private static MiniHBaseCluster cluster;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/mapreduce/TestMobSweepMapper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/mapreduce/TestMobSweepMapper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/mapreduce/TestMobSweepMapper.java
index 5ae02e4..cec8a74 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/mapreduce/TestMobSweepMapper.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/mapreduce/TestMobSweepMapper.java
@@ -49,13 +49,11 @@ import org.mockito.stubbing.Answer;
 
 @Category(SmallTests.class)
 public class TestMobSweepMapper {
-
   private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
-    TEST_UTIL.getConfiguration().setInt("hfile.format.version", 3);
-    TEST_UTIL.startMiniCluster(1);
+    TEST_UTIL.startMiniCluster();
   }
 
   @AfterClass
@@ -93,7 +91,7 @@ public class TestMobSweepMapper {
     lock.acquire();
     try {
       Mapper<ImmutableBytesWritable, Result, Text, KeyValue>.Context ctx =
-        mock(Mapper.Context.class);
+          mock(Mapper.Context.class);
       when(ctx.getConfiguration()).thenReturn(configuration);
       SweepMapper map = new SweepMapper();
       doAnswer(new Answer<Void>() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java
index 6e68201..848010b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java
@@ -62,6 +62,7 @@ import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.io.hfile.HFileContext;
 import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
+import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionRequest;
@@ -198,19 +199,20 @@ public class TestHRegionServerBulkLoad {
       }
 
       // bulk load HFiles
-      final ClusterConnection conn = (ClusterConnection) UTIL.getAdmin().getConnection();
+      final ClusterConnection conn = (ClusterConnection)UTIL.getConnection();
       Table table = conn.getTable(tableName);
-      final String bulkToken = new SecureBulkLoadClient(table).prepareBulkLoad(conn);
-      RegionServerCallable<Void> callable =
-          new RegionServerCallable<Void>(conn, tableName, Bytes.toBytes("aaa")) {
+      final String bulkToken = new SecureBulkLoadClient(UTIL.getConfiguration(), table).
+          prepareBulkLoad(conn);
+      RegionServerCallable<Void> callable = new RegionServerCallable<Void>(conn,
+          new RpcControllerFactory(UTIL.getConfiguration()), tableName, Bytes.toBytes("aaa")) {
         @Override
-        public Void call(int callTimeout) throws Exception {
+        public Void rpcCall() throws Exception {
           LOG.debug("Going to connect to server " + getLocation() + " for row "
               + Bytes.toStringBinary(getRow()));
           SecureBulkLoadClient secureClient = null;
           byte[] regionName = getLocation().getRegionInfo().getRegionName();
           try (Table table = conn.getTable(getTableName())) {
-            secureClient = new SecureBulkLoadClient(table);
+            secureClient = new SecureBulkLoadClient(UTIL.getConfiguration(), table);
             secureClient.secureBulkLoadHFiles(getStub(), famPaths, regionName,
                   true, null, bulkToken);
           }
@@ -224,15 +226,15 @@ public class TestHRegionServerBulkLoad {
       // Periodically do compaction to reduce the number of open file handles.
       if (numBulkLoads.get() % 5 == 0) {
         // 5 * 50 = 250 open file handles!
-        callable = new RegionServerCallable<Void>(conn, tableName, Bytes.toBytes("aaa")) {
+        callable = new RegionServerCallable<Void>(conn,
+            new RpcControllerFactory(UTIL.getConfiguration()), tableName, Bytes.toBytes("aaa")) {
           @Override
-          public Void call(int callTimeout) throws Exception {
+          protected Void rpcCall() throws Exception {
             LOG.debug("compacting " + getLocation() + " for row "
                 + Bytes.toStringBinary(getRow()));
             AdminProtos.AdminService.BlockingInterface server =
               conn.getAdmin(getLocation().getServerName());
-            CompactRegionRequest request =
-              RequestConverter.buildCompactRegionRequest(
+            CompactRegionRequest request = RequestConverter.buildCompactRegionRequest(
                 getLocation().getRegionInfo().getRegionName(), true, null);
             server.compactRegion(null, request);
             numCompactions.incrementAndGet();

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoadWithOldClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoadWithOldClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoadWithOldClient.java
index d55adef..e5361a0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoadWithOldClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoadWithOldClient.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.RegionServerCallable;
 import org.apache.hadoop.hbase.client.RpcRetryingCaller;
 import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
+import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionRequest;
@@ -44,6 +45,7 @@ import org.apache.hadoop.hbase.util.Pair;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
+
 import com.google.common.collect.Lists;
 
 /**
@@ -89,10 +91,12 @@ public class TestHRegionServerBulkLoadWithOldClient extends TestHRegionServerBul
 
       // bulk load HFiles
       final ClusterConnection conn = (ClusterConnection) UTIL.getAdmin().getConnection();
+      RpcControllerFactory rpcControllerFactory = new RpcControllerFactory(UTIL.getConfiguration());
       RegionServerCallable<Void> callable =
-          new RegionServerCallable<Void>(conn, tableName, Bytes.toBytes("aaa")) {
+          new RegionServerCallable<Void>(conn, rpcControllerFactory, tableName,
+              Bytes.toBytes("aaa")) {
         @Override
-        public Void call(int callTimeout) throws Exception {
+        protected Void rpcCall() throws Exception {
           LOG.info("Non-secure old client");
           byte[] regionName = getLocation().getRegionInfo().getRegionName();
               BulkLoadHFileRequest request =
@@ -109,9 +113,10 @@ public class TestHRegionServerBulkLoadWithOldClient extends TestHRegionServerBul
       // Periodically do compaction to reduce the number of open file handles.
       if (numBulkLoads.get() % 5 == 0) {
         // 5 * 50 = 250 open file handles!
-        callable = new RegionServerCallable<Void>(conn, tableName, Bytes.toBytes("aaa")) {
+        callable = new RegionServerCallable<Void>(conn, rpcControllerFactory, tableName,
+            Bytes.toBytes("aaa")) {
           @Override
-          public Void call(int callTimeout) throws Exception {
+          protected Void rpcCall() throws Exception {
             LOG.debug("compacting " + getLocation() + " for row "
                 + Bytes.toStringBinary(getRow()));
             AdminProtos.AdminService.BlockingInterface server =

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoadWithOldSecureEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoadWithOldSecureEndpoint.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoadWithOldSecureEndpoint.java
index 6de6261..f337be5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoadWithOldSecureEndpoint.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoadWithOldSecureEndpoint.java
@@ -33,13 +33,12 @@ import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.RegionServerCallable;
 import org.apache.hadoop.hbase.client.RpcRetryingCaller;
 import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
-import org.apache.hadoop.hbase.client.SecureBulkLoadClient;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
+import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionRequest;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequest;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -62,7 +61,8 @@ public class TestHRegionServerBulkLoadWithOldSecureEndpoint extends TestHRegionS
     super(duration);
   }
 
-  private static final Log LOG = LogFactory.getLog(TestHRegionServerBulkLoadWithOldSecureEndpoint.class);
+  private static final Log LOG =
+      LogFactory.getLog(TestHRegionServerBulkLoadWithOldSecureEndpoint.class);
 
   @BeforeClass
   public static void setUpBeforeClass() throws IOException {
@@ -103,16 +103,17 @@ public class TestHRegionServerBulkLoadWithOldSecureEndpoint extends TestHRegionS
       final ClusterConnection conn = (ClusterConnection) UTIL.getAdmin().getConnection();
       Table table = conn.getTable(tableName);
       final String bulkToken = new SecureBulkLoadEndpointClient(table).prepareBulkLoad(tableName);
+      RpcControllerFactory rpcControllerFactory = new RpcControllerFactory(UTIL.getConfiguration());
       RegionServerCallable<Void> callable =
-          new RegionServerCallable<Void>(conn, tableName, Bytes.toBytes("aaa")) {
+          new RegionServerCallable<Void>(conn, rpcControllerFactory, tableName,
+              Bytes.toBytes("aaa")) {
             @Override
-            public Void call(int callTimeout) throws Exception {
-              LOG.debug("Going to connect to server " + getLocation() + " for row "
-                  + Bytes.toStringBinary(getRow()));
+            protected Void rpcCall() throws Exception {
+              LOG.debug("Going to connect to server " + getLocation() + " for row " +
+                  Bytes.toStringBinary(getRow()));
               try (Table table = conn.getTable(getTableName())) {
-                boolean loaded =
-                    new SecureBulkLoadEndpointClient(table).bulkLoadHFiles(famPaths, null,
-                      bulkToken, getLocation().getRegionInfo().getStartKey());
+                boolean loaded = new SecureBulkLoadEndpointClient(table).bulkLoadHFiles(famPaths,
+                    null, bulkToken, getLocation().getRegionInfo().getStartKey());
               }
               return null;
             }
@@ -124,9 +125,10 @@ public class TestHRegionServerBulkLoadWithOldSecureEndpoint extends TestHRegionS
       // Periodically do compaction to reduce the number of open file handles.
       if (numBulkLoads.get() % 5 == 0) {
         // 5 * 50 = 250 open file handles!
-        callable = new RegionServerCallable<Void>(conn, tableName, Bytes.toBytes("aaa")) {
+        callable = new RegionServerCallable<Void>(conn, rpcControllerFactory, tableName,
+            Bytes.toBytes("aaa")) {
           @Override
-          public Void call(int callTimeout) throws Exception {
+          protected Void rpcCall() throws Exception {
             LOG.debug("compacting " + getLocation() + " for row "
                 + Bytes.toStringBinary(getRow()));
             AdminProtos.AdminService.BlockingInterface server =

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerHeartbeatMessages.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerHeartbeatMessages.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerHeartbeatMessages.java
index b906e84..2d9ba6e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerHeartbeatMessages.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerHeartbeatMessages.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.regionserver;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
@@ -33,6 +32,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.CoordinatedStateManager;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
@@ -56,7 +56,6 @@ import org.apache.hadoop.hbase.regionserver.HRegion.RegionScannerImpl;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.wal.WAL;
-import org.apache.hadoop.hbase.CellComparator;
 import org.apache.log4j.Level;
 import org.junit.After;
 import org.junit.AfterClass;

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/SparkSQLPushDownFilter.java
----------------------------------------------------------------------
diff --git a/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/SparkSQLPushDownFilter.java b/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/SparkSQLPushDownFilter.java
index fa66d69..3e90fe1 100644
--- a/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/SparkSQLPushDownFilter.java
+++ b/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/SparkSQLPushDownFilter.java
@@ -17,8 +17,6 @@
 
 package org.apache.hadoop.hbase.spark;
 
-import com.google.protobuf.ByteString;
-import com.google.protobuf.InvalidProtocolBufferException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.Cell;
@@ -37,6 +35,8 @@ import java.io.IOException;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import com.google.protobuf.InvalidProtocolBufferException;
+import com.google.protobuf.ByteString;
 
 /**
  * This filter will push down all qualifier logic given to us


[50/50] [abbrv] hbase git commit: HBASE-16418 Reduce duration of sleep waiting for region reopen in IntegrationTestBulkLoad#installSlowingCoproc()

Posted by sy...@apache.org.
HBASE-16418 Reduce duration of sleep waiting for region reopen in IntegrationTestBulkLoad#installSlowingCoproc()


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

Branch: refs/heads/hbase-12439
Commit: d5080e82fb47b5499b72fbafbbc52f4f432622d3
Parents: 9219d68
Author: tedyu <yu...@gmail.com>
Authored: Mon Aug 15 17:26:11 2016 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Mon Aug 15 17:26:11 2016 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hbase/mapreduce/IntegrationTestBulkLoad.java    | 3 ---
 1 file changed, 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/d5080e82/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestBulkLoad.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestBulkLoad.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestBulkLoad.java
index 1d8be99..4e85425 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestBulkLoad.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestBulkLoad.java
@@ -205,9 +205,6 @@ public class IntegrationTestBulkLoad extends IntegrationTestBase {
     HTableDescriptor desc = admin.getTableDescriptor(t);
     desc.addCoprocessor(SlowMeCoproScanOperations.class.getName());
     HBaseTestingUtility.modifyTableSync(admin, desc);
-    //sleep for sometime. Hope is that the regions are closed/opened before
-    //the sleep returns. TODO: do this better
-    Thread.sleep(30000);
   }
 
   @Test


[30/50] [abbrv] hbase git commit: HBASE-9465 Push entries to peer clusters serially

Posted by sy...@apache.org.
HBASE-9465 Push entries to peer clusters serially

Signed-off-by: zhangduo <zh...@apache.org>


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

Branch: refs/heads/hbase-12439
Commit: 5cadcd59aa57c9566349dc8551c958dc974e774e
Parents: 1ecb0fc
Author: Phil Yang <ud...@gmail.com>
Authored: Thu Aug 4 10:11:56 2016 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Tue Aug 9 15:25:50 2016 +0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/HTableDescriptor.java   |  12 +
 .../apache/hadoop/hbase/MetaTableAccessor.java  | 234 ++++++++++-
 .../hbase/client/ConnectionImplementation.java  |   1 +
 .../client/replication/ReplicationAdmin.java    |  14 +-
 .../org/apache/hadoop/hbase/HConstants.java     |  26 ++
 .../src/main/resources/hbase-default.xml        |  14 +
 .../hbase/protobuf/generated/WALProtos.java     |  18 +-
 hbase-protocol/src/main/protobuf/WAL.proto      |   1 +
 .../org/apache/hadoop/hbase/master/HMaster.java |   5 +
 .../hadoop/hbase/master/RegionStateStore.java   |  47 ++-
 .../master/cleaner/ReplicationMetaCleaner.java  | 186 +++++++++
 .../hbase/regionserver/wal/FSWALEntry.java      |   1 -
 .../replication/regionserver/Replication.java   |  12 +
 .../regionserver/ReplicationSource.java         | 127 +++++-
 .../regionserver/ReplicationSourceManager.java  |  87 +++-
 .../hadoop/hbase/util/FSTableDescriptors.java   |  24 ++
 .../java/org/apache/hadoop/hbase/wal/WAL.java   |  16 +
 .../hadoop/hbase/TestMetaTableAccessor.java     |   8 +-
 .../master/TestAssignmentManagerOnCluster.java  |   2 +-
 .../replication/TestSerialReplication.java      | 399 +++++++++++++++++++
 20 files changed, 1176 insertions(+), 58 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/5cadcd59/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
index ccad414..9abdf42 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
@@ -1115,6 +1115,18 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
   }
 
   /**
+   * Return true if there are at least one cf whose replication scope is serial.
+   */
+  public boolean hasSerialReplicationScope() {
+    for (HColumnDescriptor column: getFamilies()){
+      if (column.getScope() == HConstants.REPLICATION_SCOPE_SERIAL){
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
    * Returns the configured replicas per region
    */
   public int getRegionReplication() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/5cadcd59/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
index a5dbc94..1eaa753 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
@@ -17,11 +17,15 @@
  */
 package org.apache.hadoop.hbase;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.ServiceException;
+
 import java.io.Closeable;
 import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.HashMap;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
@@ -34,8 +38,6 @@ import java.util.regex.Pattern;
 
 import edu.umd.cs.findbugs.annotations.NonNull;
 import edu.umd.cs.findbugs.annotations.Nullable;
-import com.google.common.annotations.VisibleForTesting;
-import com.google.protobuf.ServiceException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -113,14 +115,31 @@ public class MetaTableAccessor {
    *                             region is the result of a merge
    * info:mergeB              => contains a serialized HRI for the second parent region if the
    *                             region is the result of a merge
-   *
    * The actual layout of meta should be encapsulated inside MetaTableAccessor methods,
    * and should not leak out of it (through Result objects, etc)
+   *
+   * For replication serially, there are two column families "rep_barrier", "rep_position" whose
+   * row key is encodedRegionName.
+   * rep_barrier:{seqid}      => in each time a RS opens a region, it saves the open sequence
+   *                                  id in this region
+   * rep_position:{peerid}    => to save the max sequence id we have pushed for each peer
+   * rep_position:_TABLENAME_ => a special cell to save this region's table name, will used when
+   *                             we clean old data
+   * rep_position:_DAUGHTER_  => a special cell to present this region is split or merged, in this
+   *                             cell the value is merged encoded name or two split encoded names
+   *                             separated by ","
    */
 
   private static final Log LOG = LogFactory.getLog(MetaTableAccessor.class);
   private static final Log METALOG = LogFactory.getLog("org.apache.hadoop.hbase.META");
 
+  // Save its daughter region(s) when split/merge
+  private static final byte[] daughterNamePosCq = Bytes.toBytes("_DAUGHTER_");
+
+  // Save its table name because we only know region's encoded name
+  private static final String tableNamePeer = "_TABLENAME_";
+  private static final byte[] tableNamePosCq = Bytes.toBytes(tableNamePeer);
+
   static final byte [] META_REGION_PREFIX;
   static {
     // Copy the prefix from FIRST_META_REGIONINFO into META_REGION_PREFIX.
@@ -1318,6 +1337,19 @@ public class MetaTableAccessor {
     return delete;
   }
 
+  public static Put makeBarrierPut(byte[] encodedRegionName, long seq, byte[] tableName) {
+    byte[] seqBytes = Bytes.toBytes(seq);
+    return new Put(encodedRegionName)
+        .addImmutable(HConstants.REPLICATION_BARRIER_FAMILY, seqBytes, seqBytes)
+        .addImmutable(HConstants.REPLICATION_POSITION_FAMILY, tableNamePosCq, tableName);
+  }
+
+
+  public static Put makeSerialDaughterPut(byte[] encodedRegionName, byte[] value) {
+    return new Put(encodedRegionName).addImmutable(HConstants.REPLICATION_POSITION_FAMILY,
+        daughterNamePosCq, value);
+  }
+
   /**
    * Adds split daughters to the Put
    */
@@ -1334,27 +1366,28 @@ public class MetaTableAccessor {
   }
 
   /**
-   * Put the passed <code>p</code> to the <code>hbase:meta</code> table.
+   * Put the passed <code>puts</code> to the <code>hbase:meta</code> table.
+   * Non-atomic for multi puts.
    * @param connection connection we're using
-   * @param p Put to add to hbase:meta
+   * @param puts Put to add to hbase:meta
    * @throws IOException
    */
-  static void putToMetaTable(final Connection connection, final Put p)
+  static void putToMetaTable(final Connection connection, final Put... puts)
     throws IOException {
-    put(getMetaHTable(connection), p);
+    put(getMetaHTable(connection), Arrays.asList(puts));
   }
 
   /**
    * @param t Table to use (will be closed when done).
-   * @param p put to make
+   * @param puts puts to make
    * @throws IOException
    */
-  private static void put(final Table t, final Put p) throws IOException {
+  private static void put(final Table t, final List<Put> puts) throws IOException {
     try {
       if (METALOG.isDebugEnabled()) {
-        METALOG.debug(mutationToString(p));
+        METALOG.debug(mutationsToString(puts));
       }
-      t.put(p);
+      t.put(puts);
     } finally {
       t.close();
     }
@@ -1490,7 +1523,7 @@ public class MetaTableAccessor {
    * Adds a (single) hbase:meta row for the specified new region and its daughters. Note that this
    * does not add its daughter's as different rows, but adds information about the daughters
    * in the same row as the parent. Use
-   * {@link #splitRegion(Connection, HRegionInfo, HRegionInfo, HRegionInfo, ServerName, int)}
+   * {@link #splitRegion(Connection, HRegionInfo, HRegionInfo, HRegionInfo, ServerName,int,boolean)}
    * if you want to do that.
    * @param meta the Table for META
    * @param regionInfo region information
@@ -1515,7 +1548,7 @@ public class MetaTableAccessor {
    * Adds a (single) hbase:meta row for the specified new region and its daughters. Note that this
    * does not add its daughter's as different rows, but adds information about the daughters
    * in the same row as the parent. Use
-   * {@link #splitRegion(Connection, HRegionInfo, HRegionInfo, HRegionInfo, ServerName, int)}
+   * {@link #splitRegion(Connection, HRegionInfo, HRegionInfo, HRegionInfo, ServerName,int,boolean)}
    * if you want to do that.
    * @param connection connection we're using
    * @param regionInfo region information
@@ -1601,11 +1634,12 @@ public class MetaTableAccessor {
    * @param regionB
    * @param sn the location of the region
    * @param masterSystemTime
+   * @param saveBarrier true if need save replication barrier in meta, used for serial replication
    * @throws IOException
    */
   public static void mergeRegions(final Connection connection, HRegionInfo mergedRegion,
       HRegionInfo regionA, HRegionInfo regionB, ServerName sn, int regionReplication,
-      long masterSystemTime)
+      long masterSystemTime, boolean saveBarrier)
           throws IOException {
     Table meta = getMetaHTable(connection);
     try {
@@ -1636,7 +1670,17 @@ public class MetaTableAccessor {
 
       byte[] tableRow = Bytes.toBytes(mergedRegion.getRegionNameAsString()
         + HConstants.DELIMITER);
-      multiMutate(meta, tableRow, putOfMerged, deleteA, deleteB);
+      Mutation[] mutations;
+      if (saveBarrier) {
+        Put putBarrierA = makeSerialDaughterPut(regionA.getEncodedNameAsBytes(),
+            Bytes.toBytes(mergedRegion.getEncodedName()));
+        Put putBarrierB = makeSerialDaughterPut(regionB.getEncodedNameAsBytes(),
+            Bytes.toBytes(mergedRegion.getEncodedName()));
+        mutations = new Mutation[] { putOfMerged, deleteA, deleteB, putBarrierA, putBarrierB };
+      } else {
+        mutations = new Mutation[] { putOfMerged, deleteA, deleteB };
+      }
+      multiMutate(meta, tableRow, mutations);
     } finally {
       meta.close();
     }
@@ -1652,10 +1696,11 @@ public class MetaTableAccessor {
    * @param splitA Split daughter region A
    * @param splitB Split daughter region A
    * @param sn the location of the region
+   * @param saveBarrier true if need save replication barrier in meta, used for serial replication
    */
-  public static void splitRegion(final Connection connection,
-                                 HRegionInfo parent, HRegionInfo splitA, HRegionInfo splitB,
-                                 ServerName sn, int regionReplication) throws IOException {
+  public static void splitRegion(final Connection connection, HRegionInfo parent,
+      HRegionInfo splitA, HRegionInfo splitB, ServerName sn, int regionReplication,
+      boolean saveBarrier) throws IOException {
     Table meta = getMetaHTable(connection);
     try {
       HRegionInfo copyOfParent = new HRegionInfo(parent);
@@ -1680,8 +1725,17 @@ public class MetaTableAccessor {
         addEmptyLocation(putB, i);
       }
 
+      Mutation[] mutations;
+      if (saveBarrier) {
+        Put putBarrier = makeSerialDaughterPut(parent.getEncodedNameAsBytes(),
+            Bytes
+                .toBytes(splitA.getEncodedName() + HConstants.DELIMITER + splitB.getEncodedName()));
+        mutations = new Mutation[]{putParent, putA, putB, putBarrier};
+      } else {
+        mutations = new Mutation[]{putParent, putA, putB};
+      }
       byte[] tableRow = Bytes.toBytes(parent.getRegionNameAsString() + HConstants.DELIMITER);
-      multiMutate(meta, tableRow, putParent, putA, putB);
+      multiMutate(meta, tableRow, mutations);
     } finally {
       meta.close();
     }
@@ -1781,6 +1835,27 @@ public class MetaTableAccessor {
   }
 
   /**
+   * Updates the progress of pushing entries to peer cluster. Skip entry if value is -1.
+   * @param connection connection we're using
+   * @param peerId the peerId to push
+   * @param positions map that saving positions for each region
+   * @throws IOException
+   */
+  public static void updateReplicationPositions(Connection connection, String peerId,
+      Map<String, Long> positions) throws IOException {
+    List<Put> puts = new ArrayList<>();
+    for (Map.Entry<String, Long> entry : positions.entrySet()) {
+      long value = Math.abs(entry.getValue());
+      Put put = new Put(Bytes.toBytes(entry.getKey()));
+      put.addImmutable(HConstants.REPLICATION_POSITION_FAMILY, Bytes.toBytes(peerId),
+          Bytes.toBytes(value));
+      puts.add(put);
+    }
+    getMetaHTable(connection).put(puts);
+  }
+
+
+  /**
    * Updates the location of the specified region to be the specified server.
    * <p>
    * Connects to the specified server which should be hosting the specified
@@ -1977,4 +2052,125 @@ public class MetaTableAccessor {
   private static String mutationToString(Mutation p) throws IOException {
     return p.getClass().getSimpleName() + p.toJSON();
   }
+
+  /**
+   * Get replication position for a peer in a region.
+   * @param connection connection we're using
+   * @return the position of this peer, -1 if no position in meta.
+   */
+  public static long getReplicationPositionForOnePeer(Connection connection,
+      byte[] encodedRegionName, String peerId) throws IOException {
+    Get get = new Get(encodedRegionName);
+    get.addColumn(HConstants.REPLICATION_POSITION_FAMILY, Bytes.toBytes(peerId));
+    Result r = get(getMetaHTable(connection), get);
+    if (r.isEmpty()) {
+      return -1;
+    }
+    Cell cell = r.rawCells()[0];
+    return Bytes.toLong(cell.getValueArray(),cell.getValueOffset(),cell.getValueLength());
+  }
+
+  /**
+   * Get replication positions for all peers in a region.
+   * @param connection connection we're using
+   * @param encodedRegionName region's encoded name
+   * @return the map of positions for each peer
+   */
+  public static Map<String, Long> getReplicationPositionForAllPeer(Connection connection,
+      byte[] encodedRegionName) throws IOException {
+    Get get = new Get(encodedRegionName);
+    get.addFamily(HConstants.REPLICATION_POSITION_FAMILY);
+    Result r = get(getMetaHTable(connection), get);
+    Map<String, Long> map = new HashMap<>((int) (r.size() / 0.75 + 1));
+    for (Cell c : r.listCells()) {
+      if (!Bytes.equals(tableNamePosCq, 0, tableNamePosCq.length, c.getQualifierArray(),
+          c.getQualifierOffset(), c.getQualifierLength()) &&
+          !Bytes.equals(daughterNamePosCq, 0, daughterNamePosCq.length, c.getQualifierArray(),
+          c.getQualifierOffset(), c.getQualifierLength())) {
+        map.put(
+            Bytes.toString(c.getQualifierArray(), c.getQualifierOffset(), c.getQualifierLength()),
+            Bytes.toLong(c.getValueArray(), c.getValueOffset(), c.getValueLength()));
+      }
+    }
+    return map;
+  }
+
+  /**
+   * Get replication barriers for all peers in a region.
+   * @param encodedRegionName region's encoded name
+   * @return a list of barrier sequence numbers.
+   * @throws IOException
+   */
+  public static List<Long> getReplicationBarriers(Connection connection, byte[] encodedRegionName)
+      throws IOException {
+    Get get = new Get(encodedRegionName);
+    get.addFamily(HConstants.REPLICATION_BARRIER_FAMILY);
+    Result r = get(getMetaHTable(connection), get);
+    List<Long> list = new ArrayList<>();
+    if (!r.isEmpty()) {
+      for (Cell cell : r.rawCells()) {
+        list.add(Bytes.toLong(cell.getQualifierArray(), cell.getQualifierOffset(),
+            cell.getQualifierLength()));
+      }
+    }
+    return list;
+  }
+
+  /**
+   * Get all barriers in all regions.
+   * @return a map of barrier lists in all regions
+   * @throws IOException
+   */
+  public static Map<String, List<Long>> getAllBarriers(Connection connection) throws IOException {
+    Map<String, List<Long>> map = new HashMap<>();
+    Scan scan = new Scan();
+    scan.addFamily(HConstants.REPLICATION_BARRIER_FAMILY);
+    try (Table t = getMetaHTable(connection);
+        ResultScanner scanner = t.getScanner(scan)) {
+      Result result;
+      while ((result = scanner.next()) != null) {
+        String key = Bytes.toString(result.getRow());
+        List<Long> list = new ArrayList<>();
+        for (Cell cell : result.rawCells()) {
+          list.add(Bytes.toLong(cell.getQualifierArray(), cell.getQualifierOffset(),
+              cell.getQualifierLength()));
+        }
+        map.put(key, list);
+      }
+    }
+    return map;
+  }
+
+  /**
+   * Get daughter region(s) for a region, only used in serial replication.
+   * @throws IOException
+   */
+  public static String getSerialReplicationDaughterRegion(Connection connection, byte[] encodedName)
+      throws IOException {
+    Get get = new Get(encodedName);
+    get.addColumn(HConstants.REPLICATION_POSITION_FAMILY, daughterNamePosCq);
+    Result result = get(getMetaHTable(connection), get);
+    if (!result.isEmpty()) {
+      Cell c = result.rawCells()[0];
+      return Bytes.toString(c.getValueArray(), c.getValueOffset(), c.getValueLength());
+    }
+    return null;
+  }
+
+  /**
+   * Get the table name for a region, only used in serial replication.
+   * @throws IOException
+   */
+  public static String getSerialReplicationTableName(Connection connection, byte[] encodedName)
+      throws IOException {
+    Get get = new Get(encodedName);
+    get.addColumn(HConstants.REPLICATION_POSITION_FAMILY, tableNamePosCq);
+    Result result = get(getMetaHTable(connection), get);
+    if (!result.isEmpty()) {
+      Cell c = result.rawCells()[0];
+      return Bytes.toString(c.getValueArray(), c.getValueOffset(), c.getValueLength());
+    }
+    return null;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/5cadcd59/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
index 04edd25..37c62c5 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
@@ -721,6 +721,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
     Scan s = new Scan();
     s.setReversed(true);
     s.setStartRow(metaKey);
+    s.addFamily(HConstants.CATALOG_FAMILY);
     s.setSmall(true);
     s.setCaching(1);
     if (this.useMetaReplicas) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/5cadcd59/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java
index dca1821..ee26e38 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java
@@ -92,8 +92,10 @@ public class ReplicationAdmin implements Closeable {
   // only Global for now, can add other type
   // such as, 1) no global replication, or 2) the table is replicated to this cluster, etc.
   public static final String REPLICATIONTYPE = "replicationType";
-  public static final String REPLICATIONGLOBAL = Integer
-      .toString(HConstants.REPLICATION_SCOPE_GLOBAL);
+  public static final String REPLICATIONGLOBAL =
+      Integer.toString(HConstants.REPLICATION_SCOPE_GLOBAL);
+  public static final String REPLICATIONSERIAL =
+      Integer.toString(HConstants.REPLICATION_SCOPE_SERIAL);
 
   private final Connection connection;
   // TODO: replication should be managed by master. All the classes except ReplicationAdmin should
@@ -430,7 +432,10 @@ public class ReplicationAdmin implements Closeable {
           HashMap<String, String> replicationEntry = new HashMap<String, String>();
           replicationEntry.put(TNAME, tableName);
           replicationEntry.put(CFNAME, column.getNameAsString());
-          replicationEntry.put(REPLICATIONTYPE, REPLICATIONGLOBAL);
+          replicationEntry.put(REPLICATIONTYPE,
+              column.getScope() == HConstants.REPLICATION_SCOPE_GLOBAL ?
+                  REPLICATIONGLOBAL :
+                  REPLICATIONSERIAL);
           replicationColFams.add(replicationEntry);
         }
       }
@@ -616,7 +621,8 @@ public class ReplicationAdmin implements Closeable {
    */
   private boolean isTableRepEnabled(HTableDescriptor htd) {
     for (HColumnDescriptor hcd : htd.getFamilies()) {
-      if (hcd.getScope() != HConstants.REPLICATION_SCOPE_GLOBAL) {
+      if (hcd.getScope() != HConstants.REPLICATION_SCOPE_GLOBAL
+          && hcd.getScope() != HConstants.REPLICATION_SCOPE_SERIAL) {
         return false;
       }
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/5cadcd59/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 ce18ef5..4c499a2 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
@@ -429,6 +429,20 @@ public final class HConstants {
   /** The catalog family */
   public static final byte [] CATALOG_FAMILY = Bytes.toBytes(CATALOG_FAMILY_STR);
 
+  /** The replication barrier family as a string*/
+  public static final String REPLICATION_BARRIER_FAMILY_STR = "rep_barrier";
+
+  /** The replication barrier family */
+  public static final byte [] REPLICATION_BARRIER_FAMILY =
+      Bytes.toBytes(REPLICATION_BARRIER_FAMILY_STR);
+
+  /** The replication barrier family as a string*/
+  public static final String REPLICATION_POSITION_FAMILY_STR = "rep_position";
+
+  /** The replication barrier family */
+  public static final byte [] REPLICATION_POSITION_FAMILY =
+      Bytes.toBytes(REPLICATION_POSITION_FAMILY_STR);
+
   /** The RegionInfo qualifier as a string */
   public static final String REGIONINFO_QUALIFIER_STR = "regioninfo";
 
@@ -636,6 +650,12 @@ public final class HConstants {
   public static final int REPLICATION_SCOPE_GLOBAL = 1;
 
   /**
+   * Scope tag for serially scoped data
+   * This data will be replicated to all peers by the order of sequence id.
+   */
+  public static final int REPLICATION_SCOPE_SERIAL = 2;
+
+  /**
    * Default cluster ID, cannot be used to identify a cluster so a key with
    * this value means it wasn't meant for replication.
    */
@@ -866,6 +886,12 @@ public final class HConstants {
   public static final boolean REPLICATION_BULKLOAD_ENABLE_DEFAULT = false;
   /** Replication cluster id of source cluster which uniquely identifies itself with peer cluster */
   public static final String REPLICATION_CLUSTER_ID = "hbase.replication.cluster.id";
+
+  public static final String
+      REPLICATION_SERIALLY_WAITING_KEY = "hbase.serial.replication.waitingMs";
+  public static final long
+      REPLICATION_SERIALLY_WAITING_DEFAULT = 10000;
+
   /**
    * Directory where the source cluster file system client configuration are placed which is used by
    * sink cluster to copy HFiles from source cluster file system

http://git-wip-us.apache.org/repos/asf/hbase/blob/5cadcd59/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 116c7d9..a791717 100644
--- a/hbase-common/src/main/resources/hbase-default.xml
+++ b/hbase-common/src/main/resources/hbase-default.xml
@@ -1571,6 +1571,20 @@ possible configurations would overwhelm and obscure the important.
         slave clusters. The default of 10 will rarely need to be changed.
     </description>
   </property>
+  <property>
+    <name>hbase.serial.replication.waitingMs</name>
+    <value>10000</value>
+    <description>
+      By default, in replication we can not make sure the order of operations in slave cluster is
+      same as the order in master. If set REPLICATION_SCOPE to 2, we will push edits by the order
+      of written. This configure is to set how long (in ms) we will wait before next checking if a
+      log can not push right now because there are some logs written before it have not been pushed.
+      A larger waiting will decrease the number of queries on hbase:meta but will enlarge the delay
+      of replication. This feature relies on zk-less assignment, and conflicts with distributed log
+      replay. So users must set hbase.assignment.usezk and hbase.master.distributed.log.replay to
+      false to support it.
+    </description>
+  </property>
   <!-- Static Web User Filter properties. -->
   <property>
     <description>

http://git-wip-us.apache.org/repos/asf/hbase/blob/5cadcd59/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/WALProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/WALProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/WALProtos.java
index 28f4d4b..a675b12 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/WALProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/WALProtos.java
@@ -21,6 +21,10 @@ public final class WALProtos {
      * <code>REPLICATION_SCOPE_GLOBAL = 1;</code>
      */
     REPLICATION_SCOPE_GLOBAL(1, 1),
+    /**
+     * <code>REPLICATION_SCOPE_SERIAL = 2;</code>
+     */
+    REPLICATION_SCOPE_SERIAL(2, 2),
     ;
 
     /**
@@ -31,6 +35,10 @@ public final class WALProtos {
      * <code>REPLICATION_SCOPE_GLOBAL = 1;</code>
      */
     public static final int REPLICATION_SCOPE_GLOBAL_VALUE = 1;
+    /**
+     * <code>REPLICATION_SCOPE_SERIAL = 2;</code>
+     */
+    public static final int REPLICATION_SCOPE_SERIAL_VALUE = 2;
 
 
     public final int getNumber() { return value; }
@@ -39,6 +47,7 @@ public final class WALProtos {
       switch (value) {
         case 0: return REPLICATION_SCOPE_LOCAL;
         case 1: return REPLICATION_SCOPE_GLOBAL;
+        case 2: return REPLICATION_SCOPE_SERIAL;
         default: return null;
       }
     }
@@ -12013,11 +12022,12 @@ public final class WALProtos {
       "\030\005 \003(\0132\031.hbase.pb.StoreDescriptor\022$\n\006ser" +
       "ver\030\006 \001(\0132\024.hbase.pb.ServerName\022\023\n\013regio" +
       "n_name\030\007 \001(\014\".\n\tEventType\022\017\n\013REGION_OPEN" +
-      "\020\000\022\020\n\014REGION_CLOSE\020\001\"\014\n\nWALTrailer*F\n\tSc" +
+      "\020\000\022\020\n\014REGION_CLOSE\020\001\"\014\n\nWALTrailer*d\n\tSc" +
       "opeType\022\033\n\027REPLICATION_SCOPE_LOCAL\020\000\022\034\n\030" +
-      "REPLICATION_SCOPE_GLOBAL\020\001B?\n*org.apache" +
-      ".hadoop.hbase.protobuf.generatedB\tWALPro" +
-      "tosH\001\210\001\000\240\001\001"
+      "REPLICATION_SCOPE_GLOBAL\020\001\022\034\n\030REPLICATIO" +
+      "N_SCOPE_SERIAL\020\002B?\n*org.apache.hadoop.hb" +
+      "ase.protobuf.generatedB\tWALProtosH\001\210\001\000\240\001" +
+      "\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/5cadcd59/hbase-protocol/src/main/protobuf/WAL.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/protobuf/WAL.proto b/hbase-protocol/src/main/protobuf/WAL.proto
index c1d465a..2494977 100644
--- a/hbase-protocol/src/main/protobuf/WAL.proto
+++ b/hbase-protocol/src/main/protobuf/WAL.proto
@@ -75,6 +75,7 @@ message WALKey {
 enum ScopeType {
   REPLICATION_SCOPE_LOCAL = 0;
   REPLICATION_SCOPE_GLOBAL = 1;
+  REPLICATION_SCOPE_SERIAL = 2;
 }
 
 message FamilyScope {

http://git-wip-us.apache.org/repos/asf/hbase/blob/5cadcd59/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index 5ce056d..2022c5e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -98,6 +98,7 @@ import org.apache.hadoop.hbase.master.balancer.ClusterStatusChore;
 import org.apache.hadoop.hbase.master.balancer.LoadBalancerFactory;
 import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
 import org.apache.hadoop.hbase.master.cleaner.LogCleaner;
+import org.apache.hadoop.hbase.master.cleaner.ReplicationMetaCleaner;
 import org.apache.hadoop.hbase.master.cleaner.ReplicationZKLockCleanerChore;
 import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan;
 import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;
@@ -311,6 +312,7 @@ public class HMaster extends HRegionServer implements MasterServices {
 
   CatalogJanitor catalogJanitorChore;
   private ReplicationZKLockCleanerChore replicationZKLockCleanerChore;
+  private ReplicationMetaCleaner replicationMetaCleaner;
   private LogCleaner logCleaner;
   private HFileCleaner hfileCleaner;
   private ExpiredMobFileCleanerChore expiredMobFileCleanerChore;
@@ -988,6 +990,8 @@ public class HMaster extends HRegionServer implements MasterServices {
         LOG.error("start replicationZKLockCleanerChore failed", e);
       }
     }
+    replicationMetaCleaner = new ReplicationMetaCleaner(this, this, cleanerInterval);
+    getChoreService().scheduleChore(replicationMetaCleaner);
   }
 
   @Override
@@ -1022,6 +1026,7 @@ public class HMaster extends HRegionServer implements MasterServices {
     if (this.logCleaner != null) this.logCleaner.cancel(true);
     if (this.hfileCleaner != null) this.hfileCleaner.cancel(true);
     if (this.replicationZKLockCleanerChore != null) this.replicationZKLockCleanerChore.cancel(true);
+    if (this.replicationMetaCleaner != null) this.replicationMetaCleaner.cancel(true);
     if (this.quotaManager != null) this.quotaManager.stop();
     if (this.activeMasterManager != null) this.activeMasterManager.stop();
     if (this.serverManager != null) this.serverManager.stop();

http://git-wip-us.apache.org/repos/asf/hbase/blob/5cadcd59/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStateStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStateStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStateStore.java
index 82e28df..2dbc087 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStateStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStateStore.java
@@ -17,22 +17,25 @@
  */
 package org.apache.hadoop.hbase.master;
 
+import com.google.common.base.Preconditions;
+
 import java.io.IOException;
 import java.util.Arrays;
+import java.util.List;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.RegionLocations;
-import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.master.RegionState.State;
@@ -44,8 +47,6 @@ import org.apache.hadoop.hbase.util.MultiHConnection;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.zookeeper.KeeperException;
 
-import com.google.common.base.Preconditions;
-
 /**
  * A helper to persist region state in meta. We may change this class
  * to StateStore later if we also use it to store other states in meta
@@ -60,7 +61,7 @@ public class RegionStateStore {
   private volatile Region metaRegion;
   private volatile boolean initialized;
   private MultiHConnection multiHConnection;
-  private final Server server;
+  private final MasterServices server;
 
   /**
    * Returns the {@link ServerName} from catalog table {@link Result}
@@ -130,7 +131,7 @@ public class RegionStateStore {
           State.SPLITTING_NEW, State.MERGED));
   }
 
-  RegionStateStore(final Server server) {
+  RegionStateStore(final MasterServices server) {
     this.server = server;
     initialized = false;
   }
@@ -187,31 +188,41 @@ public class RegionStateStore {
       State state = newState.getState();
 
       int replicaId = hri.getReplicaId();
-      Put put = new Put(MetaTableAccessor.getMetaKeyForRegion(hri));
+      Put metaPut = new Put(MetaTableAccessor.getMetaKeyForRegion(hri));
       StringBuilder info = new StringBuilder("Updating hbase:meta row ");
       info.append(hri.getRegionNameAsString()).append(" with state=").append(state);
       if (serverName != null && !serverName.equals(oldServer)) {
-        put.addImmutable(HConstants.CATALOG_FAMILY, getServerNameColumn(replicaId),
+        metaPut.addImmutable(HConstants.CATALOG_FAMILY, getServerNameColumn(replicaId),
           Bytes.toBytes(serverName.getServerName()));
         info.append(", sn=").append(serverName);
       }
       if (openSeqNum >= 0) {
         Preconditions.checkArgument(state == State.OPEN
           && serverName != null, "Open region should be on a server");
-        MetaTableAccessor.addLocation(put, serverName, openSeqNum, -1, replicaId);
+        MetaTableAccessor.addLocation(metaPut, serverName, openSeqNum, -1, replicaId);
         info.append(", openSeqNum=").append(openSeqNum);
         info.append(", server=").append(serverName);
       }
-      put.addImmutable(HConstants.CATALOG_FAMILY, getStateColumn(replicaId),
+      metaPut.addImmutable(HConstants.CATALOG_FAMILY, getStateColumn(replicaId),
         Bytes.toBytes(state.name()));
       LOG.info(info);
-
+      HTableDescriptor descriptor = server.getTableDescriptors().get(hri.getTable());
+      boolean serial = false;
+      if (descriptor != null) {
+        serial = server.getTableDescriptors().get(hri.getTable()).hasSerialReplicationScope();
+      }
+      boolean shouldPutBarrier = serial && state == State.OPEN;
       // Persist the state change to meta
       if (metaRegion != null) {
         try {
           // Assume meta is pinned to master.
           // At least, that's what we want.
-          metaRegion.put(put);
+          metaRegion.put(metaPut);
+          if (shouldPutBarrier) {
+            Put barrierPut = MetaTableAccessor.makeBarrierPut(hri.getEncodedNameAsBytes(),
+                openSeqNum, hri.getTable().getName());
+            metaRegion.put(barrierPut);
+          }
           return; // Done here
         } catch (Throwable t) {
           // In unit tests, meta could be moved away by intention
@@ -230,8 +241,10 @@ public class RegionStateStore {
         }
       }
       // Called when meta is not on master
-      multiHConnection.processBatchCallback(Arrays.asList(put),
-          TableName.META_TABLE_NAME, null, null);
+      List<Put> list = shouldPutBarrier ?
+          Arrays.asList(metaPut, MetaTableAccessor.makeBarrierPut(hri.getEncodedNameAsBytes(),
+              openSeqNum, hri.getTable().getName())) : Arrays.asList(metaPut);
+      multiHConnection.processBatchCallback(list, TableName.META_TABLE_NAME, null, null);
 
     } catch (IOException ioe) {
       LOG.error("Failed to persist region state " + newState, ioe);
@@ -241,12 +254,14 @@ public class RegionStateStore {
 
   void splitRegion(HRegionInfo p,
       HRegionInfo a, HRegionInfo b, ServerName sn, int regionReplication) throws IOException {
-    MetaTableAccessor.splitRegion(server.getConnection(), p, a, b, sn, regionReplication);
+    MetaTableAccessor.splitRegion(server.getConnection(), p, a, b, sn, regionReplication,
+        server.getTableDescriptors().get(p.getTable()).hasSerialReplicationScope());
   }
 
   void mergeRegions(HRegionInfo p,
       HRegionInfo a, HRegionInfo b, ServerName sn, int regionReplication) throws IOException {
     MetaTableAccessor.mergeRegions(server.getConnection(), p, a, b, sn, regionReplication,
-    		EnvironmentEdgeManager.currentTime());
+        EnvironmentEdgeManager.currentTime(),
+        server.getTableDescriptors().get(p.getTable()).hasSerialReplicationScope());
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/5cadcd59/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationMetaCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationMetaCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationMetaCleaner.java
new file mode 100644
index 0000000..e9647e8
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationMetaCleaner.java
@@ -0,0 +1,186 @@
+/**
+ * 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
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.cleaner;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.replication.ReplicationAdmin;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.hadoop.hbase.util.Bytes;
+
+/**
+ * This chore is to clean up the useless data in hbase:meta which is used by serial replication.
+ */
+@InterfaceAudience.Private
+public class ReplicationMetaCleaner extends ScheduledChore {
+
+  private static final Log LOG = LogFactory.getLog(ReplicationMetaCleaner.class);
+
+  private ReplicationAdmin replicationAdmin;
+  private MasterServices master;
+
+  public ReplicationMetaCleaner(MasterServices master, Stoppable stoppable, int period)
+      throws IOException {
+    super("ReplicationMetaCleaner", stoppable, period);
+    this.master = master;
+    replicationAdmin = new ReplicationAdmin(master.getConfiguration());
+  }
+
+  @Override
+  protected void chore() {
+    try {
+      Map<String, HTableDescriptor> tables = master.getTableDescriptors().getAllDescriptors();
+      Map<String, Set<String>> serialTables = new HashMap<>();
+      for (Map.Entry<String, HTableDescriptor> entry : tables.entrySet()) {
+        boolean hasSerialScope = false;
+        for (HColumnDescriptor column : entry.getValue().getFamilies()) {
+          if (column.getScope() == HConstants.REPLICATION_SCOPE_SERIAL) {
+            hasSerialScope = true;
+            break;
+          }
+        }
+        if (hasSerialScope) {
+          serialTables.put(entry.getValue().getTableName().getNameAsString(), new HashSet<String>());
+        }
+      }
+      if (serialTables.isEmpty()){
+        return;
+      }
+
+      Map<String, ReplicationPeerConfig> peers = replicationAdmin.listPeerConfigs();
+      for (Map.Entry<String, ReplicationPeerConfig> entry : peers.entrySet()) {
+        for (Map.Entry<TableName, List<String>> map : entry.getValue().getTableCFsMap()
+            .entrySet()) {
+          if (serialTables.containsKey(map.getKey().getNameAsString())) {
+            serialTables.get(map.getKey().getNameAsString()).add(entry.getKey());
+            break;
+          }
+        }
+      }
+
+      Map<String, List<Long>> barrierMap = MetaTableAccessor.getAllBarriers(master.getConnection());
+      for (Map.Entry<String, List<Long>> entry : barrierMap.entrySet()) {
+        String encodedName = entry.getKey();
+        byte[] encodedBytes = Bytes.toBytes(encodedName);
+        boolean canClearRegion = false;
+        Map<String, Long> posMap = MetaTableAccessor.getReplicationPositionForAllPeer(
+            master.getConnection(), encodedBytes);
+        if (posMap.isEmpty()) {
+          continue;
+        }
+
+        String tableName = MetaTableAccessor.getSerialReplicationTableName(
+            master.getConnection(), encodedBytes);
+        Set<String> confPeers = serialTables.get(tableName);
+        if (confPeers == null) {
+          // This table doesn't exist or all cf's scope is not serial any more, we can clear meta.
+          canClearRegion = true;
+        } else {
+          if (!allPeersHavePosition(confPeers, posMap)) {
+            continue;
+          }
+
+          String daughterValue = MetaTableAccessor
+              .getSerialReplicationDaughterRegion(master.getConnection(), encodedBytes);
+          if (daughterValue != null) {
+            //this region is merged or split
+            boolean allDaughterStart = true;
+            String[] daughterRegions = daughterValue.split(",");
+            for (String daughter : daughterRegions) {
+              byte[] region = Bytes.toBytes(daughter);
+              if (!MetaTableAccessor.getReplicationBarriers(
+                  master.getConnection(), region).isEmpty() &&
+                  !allPeersHavePosition(confPeers,
+                      MetaTableAccessor
+                          .getReplicationPositionForAllPeer(master.getConnection(), region))) {
+                allDaughterStart = false;
+                break;
+              }
+            }
+            if (allDaughterStart) {
+              canClearRegion = true;
+            }
+          }
+        }
+        if (canClearRegion) {
+          Delete delete = new Delete(encodedBytes);
+          delete.addFamily(HConstants.REPLICATION_POSITION_FAMILY);
+          delete.addFamily(HConstants.REPLICATION_BARRIER_FAMILY);
+          try (Table metaTable = master.getConnection().getTable(TableName.META_TABLE_NAME)) {
+            metaTable.delete(delete);
+          }
+        } else {
+
+          // Barriers whose seq is larger than min pos of all peers, and the last barrier whose seq
+          // is smaller than min pos should be kept. All other barriers can be deleted.
+
+          long minPos = Long.MAX_VALUE;
+          for (Map.Entry<String, Long> pos : posMap.entrySet()) {
+            minPos = Math.min(minPos, pos.getValue());
+          }
+          List<Long> barriers = entry.getValue();
+          int index = Collections.binarySearch(barriers, minPos);
+          if (index < 0) {
+            index = -index - 1;
+          }
+          Delete delete = new Delete(encodedBytes);
+          for (int i = 0; i < index - 1; i++) {
+            delete.addColumn(HConstants.REPLICATION_BARRIER_FAMILY, Bytes.toBytes(barriers.get(i)));
+          }
+          try (Table metaTable = master.getConnection().getTable(TableName.META_TABLE_NAME)) {
+            metaTable.delete(delete);
+          }
+        }
+
+      }
+
+    } catch (IOException e) {
+      LOG.error("Exception during cleaning up.", e);
+    }
+
+  }
+
+  private boolean allPeersHavePosition(Set<String> peers, Map<String, Long> posMap)
+      throws IOException {
+    for(String peer:peers){
+      if (!posMap.containsKey(peer)){
+        return false;
+      }
+    }
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/5cadcd59/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java
index 3449832..72474a0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java
@@ -125,7 +125,6 @@ class FSWALEntry extends Entry {
         CellUtil.setSequenceId(c, regionSequenceId);
       }
     }
-
     getKey().setWriteEntry(we);
     return regionSequenceId;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/5cadcd59/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
index 4f518bb..741065a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
@@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WALEntry;
+import org.apache.hadoop.hbase.protobuf.generated.WALProtos;
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor;
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor;
 import org.apache.hadoop.hbase.regionserver.ReplicationSinkService;
@@ -279,6 +280,17 @@ public class Replication extends WALActionsListener.Base implements
     for (Cell cell : logEdit.getCells()) {
       if (!CellUtil.matchingFamily(cell, WALEdit.METAFAMILY)) {
         foundOtherEdits = true;
+        break;
+      }
+    }
+
+    if (!foundOtherEdits && logEdit.getCells().size() > 0) {
+      WALProtos.RegionEventDescriptor maybeEvent =
+          WALEdit.getRegionEventDescriptor(logEdit.getCells().get(0));
+      if (maybeEvent != null && (maybeEvent.getEventType() ==
+          WALProtos.RegionEventDescriptor.EventType.REGION_CLOSE)) {
+        // In serially replication, we use scopes when reading close marker.
+        foundOtherEdits = true;
       }
     }
     if ((!replicationForBulkLoadEnabled && !foundOtherEdits) || logEdit.isReplay()) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/5cadcd59/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
index 2f3b2a8..ce0fb06 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
@@ -18,6 +18,10 @@
  */
 package org.apache.hadoop.hbase.replication.regionserver;
 
+import com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
 import com.google.common.collect.Lists;
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.Service;
@@ -29,8 +33,10 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Comparator;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.PriorityBlockingQueue;
@@ -48,9 +54,11 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.Stoppable;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.protobuf.generated.WALProtos;
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor;
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
@@ -102,6 +110,8 @@ public class ReplicationSource extends Thread
   private ReplicationQueueInfo replicationQueueInfo;
   // id of the peer cluster this source replicates to
   private String peerId;
+
+  String actualPeerId;
   // The manager of all sources to which we ping back our progress
   private ReplicationSourceManager manager;
   // Should we stop everything?
@@ -185,6 +195,8 @@ public class ReplicationSource extends Thread
     this.replicationQueueInfo = new ReplicationQueueInfo(peerClusterZnode);
     // ReplicationQueueInfo parses the peerId out of the znode for us
     this.peerId = this.replicationQueueInfo.getPeerId();
+    ReplicationQueueInfo replicationQueueInfo = new ReplicationQueueInfo(peerId);
+    this.actualPeerId = replicationQueueInfo.getPeerId();
     this.logQueueWarnThreshold = this.conf.getInt("replication.source.log.queue.warn", 2);
     this.replicationEndpoint = replicationEndpoint;
   }
@@ -507,6 +519,17 @@ public class ReplicationSource extends Thread
     // Current number of hfiles that we need to replicate
     private long currentNbHFiles = 0;
 
+    // Use guava cache to set ttl for each key
+    private LoadingCache<String, Boolean> canSkipWaitingSet = CacheBuilder.newBuilder()
+        .expireAfterAccess(1, TimeUnit.DAYS).build(
+        new CacheLoader<String, Boolean>() {
+          @Override
+          public Boolean load(String key) throws Exception {
+            return false;
+          }
+        }
+    );
+
     public ReplicationSourceWorkerThread(String walGroupId,
         PriorityBlockingQueue<Path> queue, ReplicationQueueInfo replicationQueueInfo,
         ReplicationSource source) {
@@ -588,9 +611,24 @@ public class ReplicationSource extends Thread
         currentNbOperations = 0;
         currentNbHFiles = 0;
         List<WAL.Entry> entries = new ArrayList<WAL.Entry>(1);
+
+        Map<String, Long> lastPositionsForSerialScope = new HashMap<>();
         currentSize = 0;
         try {
-          if (readAllEntriesToReplicateOrNextFile(currentWALisBeingWrittenTo, entries)) {
+          if (readAllEntriesToReplicateOrNextFile(currentWALisBeingWrittenTo, entries,
+              lastPositionsForSerialScope)) {
+            for (Map.Entry<String, Long> entry : lastPositionsForSerialScope.entrySet()) {
+              waitingUntilCanPush(entry);
+            }
+            try {
+              MetaTableAccessor
+                  .updateReplicationPositions(manager.getConnection(), actualPeerId,
+                      lastPositionsForSerialScope);
+            } catch (IOException e) {
+              LOG.error("updateReplicationPositions fail", e);
+              stopper.stop("updateReplicationPositions fail");
+            }
+
             continue;
           }
         } catch (IOException ioe) {
@@ -626,15 +664,30 @@ public class ReplicationSource extends Thread
             LOG.warn("Unable to finalize the tailing of a file", e);
           }
         }
-
+        for(Map.Entry<String, Long> entry: lastPositionsForSerialScope.entrySet()) {
+          waitingUntilCanPush(entry);
+        }
         // If we didn't get anything to replicate, or if we hit a IOE,
         // wait a bit and retry.
         // But if we need to stop, don't bother sleeping
         if (isWorkerActive() && (gotIOE || entries.isEmpty())) {
           if (this.lastLoggedPosition != this.repLogReader.getPosition()) {
-            manager.logPositionAndCleanOldLogs(this.currentPath,
-                peerClusterZnode, this.repLogReader.getPosition(),
+
+            // Save positions to meta table before zk.
+            if (!gotIOE) {
+              try {
+                MetaTableAccessor.updateReplicationPositions(manager.getConnection(), actualPeerId,
+                    lastPositionsForSerialScope);
+              } catch (IOException e) {
+                LOG.error("updateReplicationPositions fail", e);
+                stopper.stop("updateReplicationPositions fail");
+              }
+            }
+
+            manager.logPositionAndCleanOldLogs(this.currentPath, peerClusterZnode,
+                this.repLogReader.getPosition(),
                 this.replicationQueueInfo.isQueueRecovered(), currentWALisBeingWrittenTo);
+
             this.lastLoggedPosition = this.repLogReader.getPosition();
           }
           // Reset the sleep multiplier if nothing has actually gone wrong
@@ -649,8 +702,7 @@ public class ReplicationSource extends Thread
           }
           continue;
         }
-        sleepMultiplier = 1;
-        shipEdits(currentWALisBeingWrittenTo, entries);
+        shipEdits(currentWALisBeingWrittenTo, entries, lastPositionsForSerialScope);
       }
       if (replicationQueueInfo.isQueueRecovered()) {
         // use synchronize to make sure one last thread will clean the queue
@@ -672,16 +724,42 @@ public class ReplicationSource extends Thread
       }
     }
 
+    private void waitingUntilCanPush(Map.Entry<String, Long> entry) {
+      String key = entry.getKey();
+      long seq = entry.getValue();
+      boolean deleteKey = false;
+      if (seq <= 0) {
+        // There is a REGION_CLOSE marker, we can not continue skipping after this entry.
+        deleteKey = true;
+        seq = -seq;
+      }
+
+      if (!canSkipWaitingSet.getUnchecked(key)) {
+        try {
+          manager.waitUntilCanBePushed(Bytes.toBytes(key), seq, actualPeerId);
+        } catch (Exception e) {
+          LOG.error("waitUntilCanBePushed fail", e);
+          stopper.stop("waitUntilCanBePushed fail");
+        }
+        canSkipWaitingSet.put(key, true);
+      }
+      if (deleteKey) {
+        canSkipWaitingSet.invalidate(key);
+      }
+    }
+
     /**
      * Read all the entries from the current log files and retain those that need to be replicated.
      * Else, process the end of the current file.
      * @param currentWALisBeingWrittenTo is the current WAL being written to
      * @param entries resulting entries to be replicated
+     * @param lastPosition save the last sequenceid for each region if the table has
+     *                     serial-replication scope
      * @return true if we got nothing and went to the next file, false if we got entries
      * @throws IOException
      */
     protected boolean readAllEntriesToReplicateOrNextFile(boolean currentWALisBeingWrittenTo,
-        List<WAL.Entry> entries) throws IOException {
+        List<WAL.Entry> entries, Map<String, Long> lastPosition) throws IOException {
       long seenEntries = 0;
       if (LOG.isTraceEnabled()) {
         LOG.trace("Seeking in " + this.currentPath + " at position "
@@ -694,6 +772,27 @@ public class ReplicationSource extends Thread
         metrics.incrLogEditsRead();
         seenEntries++;
 
+        if (entry.hasSerialReplicationScope()) {
+          String key = Bytes.toString(entry.getKey().getEncodedRegionName());
+          lastPosition.put(key, entry.getKey().getSequenceId());
+          if (entry.getEdit().getCells().size() > 0) {
+            WALProtos.RegionEventDescriptor maybeEvent =
+                WALEdit.getRegionEventDescriptor(entry.getEdit().getCells().get(0));
+            if (maybeEvent != null && maybeEvent.getEventType()
+                == WALProtos.RegionEventDescriptor.EventType.REGION_CLOSE) {
+              // In serially replication, if we move a region to another RS and move it back, we may
+              // read logs crossing two sections. We should break at REGION_CLOSE and push the first
+              // section first in case of missing the middle section belonging to the other RS.
+              // In a worker thread, if we can push the first log of a region, we can push all logs
+              // in the same region without waiting until we read a close marker because next time
+              // we read logs in this region, it must be a new section and not adjacent with this
+              // region. Mark it negative.
+              lastPosition.put(key, -entry.getKey().getSequenceId());
+              break;
+            }
+          }
+        }
+
         // don't replicate if the log entries have already been consumed by the cluster
         if (replicationEndpoint.canReplicateToSameCluster()
             || !entry.getKey().getClusterIds().contains(peerClusterId)) {
@@ -723,6 +822,7 @@ public class ReplicationSource extends Thread
             || entries.size() >= replicationQueueNbCapacity) {
           break;
         }
+
         try {
           entry = this.repLogReader.readNextAndSetPosition();
         } catch (IOException ie) {
@@ -995,7 +1095,8 @@ public class ReplicationSource extends Thread
      * @param currentWALisBeingWrittenTo was the current WAL being (seemingly)
      * written to when this method was called
      */
-    protected void shipEdits(boolean currentWALisBeingWrittenTo, List<WAL.Entry> entries) {
+    protected void shipEdits(boolean currentWALisBeingWrittenTo, List<WAL.Entry> entries,
+        Map<String, Long> lastPositionsForSerialScope) {
       int sleepMultiplier = 0;
       if (entries.isEmpty()) {
         LOG.warn("Was given 0 edits to ship");
@@ -1046,6 +1147,16 @@ public class ReplicationSource extends Thread
             for (int i = 0; i < size; i++) {
               cleanUpHFileRefs(entries.get(i).getEdit());
             }
+
+            // Save positions to meta table before zk.
+            try {
+              MetaTableAccessor.updateReplicationPositions(manager.getConnection(), actualPeerId,
+                  lastPositionsForSerialScope);
+            } catch (IOException e) {
+              LOG.error("updateReplicationPositions fail", e);
+              stopper.stop("updateReplicationPositions fail");
+            }
+
             //Log and clean up WAL logs
             manager.logPositionAndCleanOldLogs(this.currentPath, peerClusterZnode,
               this.repLogReader.getPosition(), this.replicationQueueInfo.isQueueRecovered(),

http://git-wip-us.apache.org/repos/asf/hbase/blob/5cadcd59/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
index 3cb7a84..a6f1891 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
@@ -24,6 +24,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -48,10 +49,13 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.TableDescriptors;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost;
 import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
@@ -64,6 +68,7 @@ import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
 import org.apache.hadoop.hbase.replication.ReplicationQueues;
 import org.apache.hadoop.hbase.replication.ReplicationTracker;
 import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
 
 /**
@@ -118,6 +123,8 @@ public class ReplicationSourceManager implements ReplicationListener {
   private final Random rand;
   private final boolean replicationForBulkLoadDataEnabled;
 
+  private Connection connection;
+  private long replicationWaitTime;
 
   /**
    * Creates a replication manager and sets the watch on all the other registered region servers
@@ -134,7 +141,7 @@ public class ReplicationSourceManager implements ReplicationListener {
   public ReplicationSourceManager(final ReplicationQueues replicationQueues,
       final ReplicationPeers replicationPeers, final ReplicationTracker replicationTracker,
       final Configuration conf, final Server server, final FileSystem fs, final Path logDir,
-      final Path oldLogDir, final UUID clusterId) {
+      final Path oldLogDir, final UUID clusterId) throws IOException {
     //CopyOnWriteArrayList is thread-safe.
     //Generally, reading is more than modifying.
     this.sources = new CopyOnWriteArrayList<ReplicationSourceInterface>();
@@ -171,6 +178,9 @@ public class ReplicationSourceManager implements ReplicationListener {
     replicationForBulkLoadDataEnabled =
         conf.getBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY,
           HConstants.REPLICATION_BULKLOAD_ENABLE_DEFAULT);
+    this.replicationWaitTime = conf.getLong(HConstants.REPLICATION_SERIALLY_WAITING_KEY,
+          HConstants.REPLICATION_SERIALLY_WAITING_DEFAULT);
+    connection = ConnectionFactory.createConnection(conf);
   }
 
   /**
@@ -782,6 +792,10 @@ public class ReplicationSourceManager implements ReplicationListener {
     return this.fs;
   }
 
+  public Connection getConnection() {
+    return this.connection;
+  }
+
   /**
    * Get the ReplicationPeers used by this ReplicationSourceManager
    * @return the ReplicationPeers used by this ReplicationSourceManager
@@ -814,4 +828,75 @@ public class ReplicationSourceManager implements ReplicationListener {
   public void cleanUpHFileRefs(String peerId, List<String> files) {
     this.replicationQueues.removeHFileRefs(peerId, files);
   }
+
+  /**
+   * Whether an entry can be pushed to the peer or not right now.
+   * If we enable serial replication, we can not push the entry until all entries in its region
+   * whose sequence numbers are smaller than this entry have been pushed.
+   * For each ReplicationSource, we need only check the first entry in each region, as long as it
+   * can be pushed, we can push all in this ReplicationSource.
+   * This method will be blocked until we can push.
+   * @return the first barrier of entry's region, or -1 if there is no barrier. It is used to
+   *         prevent saving positions in the region of no barrier.
+   */
+  void waitUntilCanBePushed(byte[] encodedName, long seq, String peerId)
+      throws IOException, InterruptedException {
+
+    /**
+     * There are barriers for this region and position for this peer. N barriers form N intervals,
+     * (b1,b2) (b2,b3) ... (bn,max). Generally, there is no logs whose seq id is not greater than
+     * the first barrier and the last interval is start from the last barrier.
+     *
+     * There are several conditions that we can push now, otherwise we should block:
+     * 1) "Serial replication" is not enabled, we can push all logs just like before. This case
+     *    should not call this method.
+     * 2) There is no barriers for this region, or the seq id is smaller than the first barrier.
+     *    It is mainly because we alter REPLICATION_SCOPE = 2. We can not guarantee the
+     *    order of logs that is written before altering.
+     * 3) This entry is in the first interval of barriers. We can push them because it is the
+     *    start of a region. Splitting/merging regions are also ok because the first section of
+     *    daughter region is in same region of parents and the order in one RS is guaranteed.
+     * 4) If the entry's seq id and the position are in same section, or the pos is the last
+     *    number of previous section. Because when open a region we put a barrier the number
+     *    is the last log's id + 1.
+     * 5) Log's seq is smaller than pos in meta, we are retrying. It may happen when a RS crashes
+     *    after save replication meta and before save zk offset.
+     */
+    List<Long> barriers = MetaTableAccessor.getReplicationBarriers(connection, encodedName);
+    if (barriers.isEmpty() || seq <= barriers.get(0)) {
+      // Case 2
+      return;
+    }
+    int interval = Collections.binarySearch(barriers, seq);
+    if (interval < 0) {
+      interval = -interval - 1;// get the insert position if negative
+    }
+    if (interval == 1) {
+      // Case 3
+      return;
+    }
+
+    while (true) {
+      long pos = MetaTableAccessor.getReplicationPositionForOnePeer(connection, encodedName, peerId);
+      if (seq <= pos) {
+        // Case 5
+      }
+      if (pos >= 0) {
+        // Case 4
+        int posInterval = Collections.binarySearch(barriers, pos);
+        if (posInterval < 0) {
+          posInterval = -posInterval - 1;// get the insert position if negative
+        }
+        if (posInterval == interval || pos == barriers.get(interval - 1) - 1) {
+          return;
+        }
+      }
+
+      LOG.info(Bytes.toString(encodedName) + " can not start pushing to peer " + peerId
+          + " because previous log has not been pushed: sequence=" + seq + " pos=" + pos
+          + " barriers=" + Arrays.toString(barriers.toArray()));
+      Thread.sleep(replicationWaitTime);
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/5cadcd59/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java
index 1c59a44..81dadd9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java
@@ -144,6 +144,30 @@ public class FSTableDescriptors implements TableDescriptors {
                     // Enable cache of data blocks in L1 if more than one caching tier deployed:
                     // e.g. if using CombinedBlockCache (BucketCache).
                 .setCacheDataInL1(true),
+            new HColumnDescriptor(HConstants.REPLICATION_BARRIER_FAMILY)
+                .setMaxVersions(conf.getInt(HConstants.HBASE_META_VERSIONS,
+                    HConstants.DEFAULT_HBASE_META_VERSIONS))
+                .setInMemory(true)
+                .setBlocksize(conf.getInt(HConstants.HBASE_META_BLOCK_SIZE,
+                    HConstants.DEFAULT_HBASE_META_BLOCK_SIZE))
+                .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
+                // Disable blooms for meta.  Needs work.  Seems to mess w/ getClosestOrBefore.
+                .setBloomFilterType(BloomType.NONE)
+                // Enable cache of data blocks in L1 if more than one caching tier deployed:
+                // e.g. if using CombinedBlockCache (BucketCache).
+                .setCacheDataInL1(true),
+            new HColumnDescriptor(HConstants.REPLICATION_POSITION_FAMILY)
+                .setMaxVersions(conf.getInt(HConstants.HBASE_META_VERSIONS,
+                    HConstants.DEFAULT_HBASE_META_VERSIONS))
+                .setInMemory(true)
+                .setBlocksize(conf.getInt(HConstants.HBASE_META_BLOCK_SIZE,
+                    HConstants.DEFAULT_HBASE_META_BLOCK_SIZE))
+                .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
+                // Disable blooms for meta.  Needs work.  Seems to mess w/ getClosestOrBefore.
+                .setBloomFilterType(BloomType.NONE)
+                // Enable cache of data blocks in L1 if more than one caching tier deployed:
+                // e.g. if using CombinedBlockCache (BucketCache).
+                .setCacheDataInL1(true),
             new HColumnDescriptor(HConstants.TABLE_FAMILY)
                 // Ten is arbitrary number.  Keep versions to help debugging.
                 .setMaxVersions(10)

http://git-wip-us.apache.org/repos/asf/hbase/blob/5cadcd59/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java
index af63b0b..79321b3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java
@@ -22,8 +22,11 @@ package org.apache.hadoop.hbase.wal;
 import com.google.common.annotations.VisibleForTesting;
 import java.io.Closeable;
 import java.io.IOException;
+import java.util.Map;
 import java.util.Set;
 
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
@@ -35,6 +38,7 @@ import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
 import org.apache.hadoop.hbase.regionserver.wal.WALCoprocessorHost;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.apache.hadoop.hbase.util.Bytes;
 
 /**
  * A Write Ahead Log (WAL) provides service for reading, writing waledits. This interface provides
@@ -282,6 +286,18 @@ public interface WAL {
       key.setCompressionContext(compressionContext);
     }
 
+    public boolean hasSerialReplicationScope () {
+      if (getKey().getReplicationScopes() == null || getKey().getReplicationScopes().isEmpty()) {
+        return false;
+      }
+      for (Map.Entry<byte[], Integer> e:getKey().getReplicationScopes().entrySet()) {
+        if (e.getValue() == HConstants.REPLICATION_SCOPE_SERIAL){
+          return true;
+        }
+      }
+      return false;
+    }
+
     @Override
     public String toString() {
       return this.key + "=" + this.edit;

http://git-wip-us.apache.org/repos/asf/hbase/blob/5cadcd59/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java
index 8b84452..d750faf 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java
@@ -452,7 +452,7 @@ public class TestMetaTableAccessor {
       List<HRegionInfo> regionInfos = Lists.newArrayList(parent);
       MetaTableAccessor.addRegionsToMeta(connection, regionInfos, 3);
 
-      MetaTableAccessor.splitRegion(connection, parent, splitA, splitB, serverName0, 3);
+      MetaTableAccessor.splitRegion(connection, parent, splitA, splitB, serverName0, 3, false);
 
       assertEmptyMetaLocation(meta, splitA.getRegionName(), 1);
       assertEmptyMetaLocation(meta, splitA.getRegionName(), 2);
@@ -481,7 +481,7 @@ public class TestMetaTableAccessor {
       MetaTableAccessor.addRegionsToMeta(connection, regionInfos, 3);
 
       MetaTableAccessor.mergeRegions(connection, merged, parentA, parentB, serverName0, 3,
-          HConstants.LATEST_TIMESTAMP);
+          HConstants.LATEST_TIMESTAMP, false);
 
       assertEmptyMetaLocation(meta, merged.getRegionName(), 1);
       assertEmptyMetaLocation(meta, merged.getRegionName(), 2);
@@ -609,7 +609,7 @@ public class TestMetaTableAccessor {
 
       // now merge the regions, effectively deleting the rows for region a and b.
       MetaTableAccessor.mergeRegions(connection, mergedRegionInfo,
-        regionInfoA, regionInfoB, sn, 1, masterSystemTime);
+        regionInfoA, regionInfoB, sn, 1, masterSystemTime, false);
 
       result = meta.get(get);
       serverCell = result.getColumnLatestCell(HConstants.CATALOG_FAMILY,
@@ -692,7 +692,7 @@ public class TestMetaTableAccessor {
       }
       SpyingRpcScheduler scheduler = (SpyingRpcScheduler) rs.getRpcServer().getScheduler();
       long prevCalls = scheduler.numPriorityCalls;
-      MetaTableAccessor.splitRegion(connection, parent, splitA, splitB, loc.getServerName(), 1);
+      MetaTableAccessor.splitRegion(connection, parent, splitA, splitB,loc.getServerName(),1,false);
 
       assertTrue(prevCalls < scheduler.numPriorityCalls);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/5cadcd59/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java
index 7d3d2e9..c15ccf4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java
@@ -1211,7 +1211,7 @@ public class TestAssignmentManagerOnCluster {
   public void testUpdatesRemoteMeta() throws Exception {
     conf.setInt("hbase.regionstatestore.meta.connection", 3);
     final RegionStateStore rss =
-        new RegionStateStore(new MyRegionServer(conf, new ZkCoordinatedStateManager()));
+        new RegionStateStore(new MyMaster(conf, new ZkCoordinatedStateManager()));
     rss.start();
     // Create 10 threads and make each do 10 puts related to region state update
     Thread[] th = new Thread[10];


[35/50] [abbrv] hbase git commit: HBASE-14345 Consolidate printUsage in IntegrationTestLoadAndVerify (Reid Chan)

Posted by sy...@apache.org.
HBASE-14345 Consolidate printUsage in IntegrationTestLoadAndVerify (Reid Chan)


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

Branch: refs/heads/hbase-12439
Commit: 3c3457c6c07674db19be392a8ddc1d2de1e0f2c8
Parents: 091a17e
Author: Reid <re...@outlook.com>
Authored: Sun Aug 7 14:12:30 2016 +0800
Committer: Apekshit Sharma <ap...@apache.org>
Committed: Wed Aug 10 09:38:46 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/test/IntegrationTestLoadAndVerify.java | 12 +++++++-----
 .../IntegrationTestWithCellVisibilityLoadAndVerify.java |  4 ++--
 2 files changed, 9 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/3c3457c6/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java
index d12383d..5b437d4 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java
@@ -549,9 +549,11 @@ public void cleanUpCluster() throws Exception {
     getTestingUtil(getConf()).deleteTable(htd.getTableName());
   }
 
-  public void usage() {
-    System.err.println(this.getClass().getSimpleName()
-      + " [-Doptions] <load|verify|loadAndVerify|search>");
+  @Override
+  public void printUsage() {
+    printUsage(this.getClass().getSimpleName() + " <options>"
+        + " [-Doptions] <load|verify|loadAndVerify|search>", "Options", "");
+    System.err.println("");
     System.err.println("  Loads a table with row dependencies and verifies the dependency chains");
     System.err.println("Options");
     System.err.println("  -Dloadmapper.table=<name>        Table to write/verify (default autogen)");
@@ -571,7 +573,7 @@ public void cleanUpCluster() throws Exception {
 
     String[] args = cmd.getArgs();
     if (args == null || args.length < 1) {
-      usage();
+      printUsage();
       throw new RuntimeException("Incorrect Number of args.");
     }
     toRun = args[0];
@@ -608,7 +610,7 @@ public void cleanUpCluster() throws Exception {
       }
     } else {
       System.err.println("Invalid argument " + toRun);
-      usage();
+      printUsage();
       return 1;
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/3c3457c6/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestWithCellVisibilityLoadAndVerify.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestWithCellVisibilityLoadAndVerify.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestWithCellVisibilityLoadAndVerify.java
index abaa481..3cafe9d 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestWithCellVisibilityLoadAndVerify.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestWithCellVisibilityLoadAndVerify.java
@@ -347,7 +347,7 @@ public class IntegrationTestWithCellVisibilityLoadAndVerify extends IntegrationT
   }
 
   @Override
-  public void usage() {
+  public void printUsage() {
     System.err.println(this.getClass().getSimpleName() + " -u usera,userb [-Doptions]");
     System.err.println("  Loads a table with cell visibilities and verifies with Authorizations");
     System.err.println("Options");
@@ -386,7 +386,7 @@ public class IntegrationTestWithCellVisibilityLoadAndVerify extends IntegrationT
   protected void processOptions(CommandLine cmd) {
     List args = cmd.getArgList();
     if (args.size() > 0) {
-      usage();
+      printUsage();
       throw new RuntimeException("No args expected.");
     }
     // We always want loadAndVerify action


[11/50] [abbrv] hbase git commit: HBASE-16340 exclude Xerces iplementation jars from coming in transitively.

Posted by sy...@apache.org.
HBASE-16340 exclude Xerces iplementation jars from coming in transitively.

Signed-off-by: Esteban Gutierrez <es...@apache.org>


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

Branch: refs/heads/hbase-12439
Commit: c59f76485e2fbb86124847c4b31a06dc53d393b8
Parents: 60b79e2
Author: Sean Busbey <bu...@apache.org>
Authored: Tue Aug 2 11:36:51 2016 -0500
Committer: Sean Busbey <bu...@apache.org>
Committed: Fri Aug 5 10:16:29 2016 -0500

----------------------------------------------------------------------
 hbase-spark/pom.xml |  4 ++++
 pom.xml             | 24 +++++++++++++++++++++++-
 2 files changed, 27 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/c59f7648/hbase-spark/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-spark/pom.xml b/hbase-spark/pom.xml
index 842ff21..aa03854 100644
--- a/hbase-spark/pom.xml
+++ b/hbase-spark/pom.xml
@@ -272,6 +272,10 @@
                     <groupId>io.netty</groupId>
                     <artifactId>netty</artifactId>
                 </exclusion>
+                <exclusion>
+                  <groupId>xerces</groupId>
+                  <artifactId>xercesImpl</artifactId>
+                </exclusion>
             </exclusions>
         </dependency>
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c59f7648/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index c4f1b05..71b77bb 100644
--- a/pom.xml
+++ b/pom.xml
@@ -933,7 +933,7 @@
         <!-- version set by parent -->
         <executions>
           <execution>
-            <id>min-maven-and-java</id>
+            <id>min-maven-min-java-banned-xerces</id>
             <goals>
               <goal>enforce</goal>
             </goals>
@@ -959,6 +959,12 @@
   See the reference guide on building for more information: http://hbase.apache.org/book.html#build
                   </message>
                 </requireJavaVersion>
+                <bannedDependencies>
+                  <excludes>
+                    <exclude>xerces:xercesImpl</exclude>
+                  </excludes>
+                  <message>We avoid adding our own Xerces jars to the classpath, see HBASE-16340.</message>
+                </bannedDependencies>
               </rules>
             </configuration>
           </execution>
@@ -2176,6 +2182,10 @@
                 <groupId>stax</groupId>
                 <artifactId>stax-api</artifactId>
               </exclusion>
+              <exclusion>
+                <groupId>xerces</groupId>
+                <artifactId>xercesImpl</artifactId>
+              </exclusion>
             </exclusions>
             <version>${hadoop-two.version}</version>
           </dependency>
@@ -2202,6 +2212,10 @@
                 <groupId>stax</groupId>
                 <artifactId>stax-api</artifactId>
               </exclusion>
+              <exclusion>
+                <groupId>xerces</groupId>
+                <artifactId>xercesImpl</artifactId>
+              </exclusion>
             </exclusions>
           </dependency>
           <dependency>
@@ -2362,6 +2376,10 @@
                <groupId>stax</groupId>
                <artifactId>stax-api</artifactId>
              </exclusion>
+             <exclusion>
+               <groupId>xerces</groupId>
+               <artifactId>xercesImpl</artifactId>
+             </exclusion>
            </exclusions>
            <version>${hadoop-three.version}</version>
          </dependency>
@@ -2384,6 +2402,10 @@
                <groupId>stax</groupId>
                <artifactId>stax-api</artifactId>
              </exclusion>
+             <exclusion>
+               <groupId>xerces</groupId>
+               <artifactId>xercesImpl</artifactId>
+             </exclusion>
            </exclusions>
          </dependency>
          <dependency>


[42/50] [abbrv] hbase git commit: HBASE-16385 Have hbase-rest pull hbase.rest.port from Constants.java (Yi Liang)

Posted by sy...@apache.org.
HBASE-16385 Have hbase-rest pull hbase.rest.port from Constants.java (Yi Liang)


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

Branch: refs/heads/hbase-12439
Commit: 88956676d7e7164f3a5b0dfbb31ad67bd3d5ed87
Parents: b3888ea
Author: stack <st...@apache.org>
Authored: Wed Aug 10 16:06:05 2016 -0700
Committer: stack <st...@apache.org>
Committed: Wed Aug 10 16:06:05 2016 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/hbase/rest/RESTServer.java   | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/88956676/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RESTServer.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RESTServer.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RESTServer.java
index cb37fb5..b8ab964 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RESTServer.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RESTServer.java
@@ -151,7 +151,7 @@ public class RESTServer implements Constants {
 
   private static void parseCommandLine(String[] args, RESTServlet servlet) {
     Options options = new Options();
-    options.addOption("p", "port", true, "Port to bind to [default: 8080]");
+    options.addOption("p", "port", true, "Port to bind to [default: " + DEFAULT_LISTEN_PORT + "]");
     options.addOption("ro", "readonly", false, "Respond only to GET HTTP " +
       "method requests [default: false]");
     options.addOption(null, "infoport", true, "Port for web UI");
@@ -263,7 +263,7 @@ public class RESTServer implements Constants {
       sslConnector.setKeyPassword(keyPassword);
       connector = sslConnector;
     }
-    connector.setPort(servlet.getConfiguration().getInt("hbase.rest.port", 8080));
+    connector.setPort(servlet.getConfiguration().getInt("hbase.rest.port", DEFAULT_LISTEN_PORT));
     connector.setHost(servlet.getConfiguration().get("hbase.rest.host", "0.0.0.0"));
     connector.setHeaderBufferSize(65536);
 


[23/50] [abbrv] hbase git commit: HBASE-16362 Mob compaction does not set cacheBlocks to false when creating StoreScanner

Posted by sy...@apache.org.
HBASE-16362 Mob compaction does not set cacheBlocks to false when creating StoreScanner


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

Branch: refs/heads/hbase-12439
Commit: 977858c9cbdcc7fbdc284e21483ba61a744d6c02
Parents: 938ee73
Author: zhangduo <zh...@apache.org>
Authored: Fri Aug 5 13:58:02 2016 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Sat Aug 6 22:38:26 2016 +0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/regionserver/StoreScanner.java   | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/977858c9/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
index 91a77ea..e008a40 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
@@ -319,7 +319,8 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
   public StoreScanner(final Scan scan, ScanInfo scanInfo, ScanType scanType,
       final NavigableSet<byte[]> columns, final List<KeyValueScanner> scanners, long earliestPutTs,
       long readPt) throws IOException {
-    this(null, scan, scanInfo, columns, readPt, scan.getCacheBlocks());
+    this(null, scan, scanInfo, columns, readPt,
+        scanType == ScanType.USER_SCAN ? scan.getCacheBlocks() : false);
     if (scanType == ScanType.USER_SCAN) {
       this.matcher = UserScanQueryMatcher.create(scan, scanInfo, columns, oldestUnexpiredTS, now,
         null);


[18/50] [abbrv] hbase git commit: Revert "HBASE-16308 Contain protobuf references Gather up the pb references into a few locations only rather than have pb references distributed all about the code base."

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/0206dc67/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/TimeLimitedRpcController.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/TimeLimitedRpcController.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/TimeLimitedRpcController.java
new file mode 100644
index 0000000..cf08ea9
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/TimeLimitedRpcController.java
@@ -0,0 +1,142 @@
+/**
+ * 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
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.ipc;
+
+import com.google.protobuf.RpcCallback;
+import com.google.protobuf.RpcController;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+@InterfaceAudience.Private
+public class TimeLimitedRpcController implements RpcController {
+
+  /**
+   * The time, in ms before the call should expire.
+   */
+  protected volatile Integer callTimeout;
+  protected volatile boolean cancelled = false;
+  protected final AtomicReference<RpcCallback<Object>> cancellationCb =
+      new AtomicReference<>(null);
+
+  protected final AtomicReference<RpcCallback<IOException>> failureCb =
+      new AtomicReference<>(null);
+
+  private IOException exception;
+
+  public int getCallTimeout() {
+    if (callTimeout != null) {
+      return callTimeout;
+    } else {
+      return 0;
+    }
+  }
+
+  public void setCallTimeout(int callTimeout) {
+    this.callTimeout = callTimeout;
+  }
+
+  public boolean hasCallTimeout(){
+    return callTimeout != null;
+  }
+
+  @Override
+  public String errorText() {
+    if (exception != null) {
+      return exception.getMessage();
+    } else {
+      return null;
+    }
+  }
+
+  /**
+   * For use in async rpc clients
+   * @return true if failed
+   */
+  @Override
+  public boolean failed() {
+    return this.exception != null;
+  }
+
+  @Override
+  public boolean isCanceled() {
+    return cancelled;
+  }
+
+  @Override
+  public void notifyOnCancel(RpcCallback<Object> cancellationCb) {
+    this.cancellationCb.set(cancellationCb);
+    if (this.cancelled) {
+      cancellationCb.run(null);
+    }
+  }
+
+  /**
+   * Notify a callback on error.
+   * For use in async rpc clients
+   *
+   * @param failureCb the callback to call on error
+   */
+  public void notifyOnFail(RpcCallback<IOException> failureCb) {
+    this.failureCb.set(failureCb);
+    if (this.exception != null) {
+      failureCb.run(this.exception);
+    }
+  }
+
+  @Override
+  public void reset() {
+    exception = null;
+    cancelled = false;
+    failureCb.set(null);
+    cancellationCb.set(null);
+    callTimeout = null;
+  }
+
+  @Override
+  public void setFailed(String reason) {
+    this.exception = new IOException(reason);
+    if (this.failureCb.get() != null) {
+      this.failureCb.get().run(this.exception);
+    }
+  }
+
+  /**
+   * Set failed with an exception to pass on.
+   * For use in async rpc clients
+   *
+   * @param e exception to set with
+   */
+  public void setFailed(IOException e) {
+    this.exception = e;
+    if (this.failureCb.get() != null) {
+      this.failureCb.get().run(this.exception);
+    }
+  }
+
+  @Override
+  public void startCancel() {
+    cancelled = true;
+    if (cancellationCb.get() != null) {
+      cancellationCb.get().run(null);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0206dc67/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
index 623acd5..5ba0572 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
@@ -17,9 +17,6 @@
  */
 package org.apache.hadoop.hbase.protobuf;
 
-import static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier
-.RegionSpecifierType.REGION_NAME;
-
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
@@ -41,11 +38,14 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.NavigableSet;
 import java.util.Set;
-import java.util.concurrent.Callable;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+
+import static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier
+.RegionSpecifierType.REGION_NAME;
+
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.CellUtil;
@@ -53,7 +53,6 @@ import org.apache.hadoop.hbase.ClusterId;
 import org.apache.hadoop.hbase.ClusterStatus;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HBaseIOException;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
@@ -125,8 +124,8 @@ import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos;
 import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.LiveServerInfo;
 import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionInTransition;
 import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionLoad;
-import org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos;
 import org.apache.hadoop.hbase.protobuf.generated.FSProtos.HBaseVersionFileContent;
+import org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos;
 import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair;
@@ -172,9 +171,11 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.DynamicClassLoader;
 import org.apache.hadoop.hbase.util.ExceptionUtil;
 import org.apache.hadoop.hbase.util.Methods;
+import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.VersionInfo;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 
 import com.google.common.collect.ArrayListMultimap;
@@ -333,32 +334,17 @@ public final class ProtobufUtil {
    *   a new IOException that wraps the unexpected ServiceException.
    */
   public static IOException getRemoteException(ServiceException se) {
-    return makeIOExceptionOfException(se);
-  }
-
-  /**
-   * Like {@link #getRemoteException(ServiceException)} but more generic, able to handle more than
-   * just {@link ServiceException}. Prefer this method to
-   * {@link #getRemoteException(ServiceException)} because trying to
-   * contain direct protobuf references.
-   * @param e
-   */
-  public static IOException handleRemoteException(Exception e) {
-    return makeIOExceptionOfException(e);
-  }
-
-  private static IOException makeIOExceptionOfException(Exception e) {
-    Throwable t = e;
-    if (e instanceof ServiceException) {
-      t = e.getCause();
+    Throwable e = se.getCause();
+    if (e == null) {
+      return new IOException(se);
     }
-    if (ExceptionUtil.isInterrupt(t)) {
-      return ExceptionUtil.asInterrupt(t);
+    if (ExceptionUtil.isInterrupt(e)) {
+      return ExceptionUtil.asInterrupt(e);
     }
-    if (t instanceof RemoteException) {
-      t = ((RemoteException)t).unwrapRemoteException();
+    if (e instanceof RemoteException) {
+      e = ((RemoteException) e).unwrapRemoteException();
     }
-    return t instanceof IOException? (IOException)t: new HBaseIOException(t);
+    return e instanceof IOException ? (IOException) e : new IOException(se);
   }
 
   /**
@@ -1266,6 +1252,7 @@ public final class ProtobufUtil {
     return toMutation(type, mutation, builder, HConstants.NO_NONCE);
   }
 
+  @SuppressWarnings("deprecation")
   public static MutationProto toMutation(final MutationType type, final Mutation mutation,
       MutationProto.Builder builder, long nonce)
   throws IOException {
@@ -2671,11 +2658,13 @@ public final class ProtobufUtil {
     }
   }
 
+  @SuppressWarnings("deprecation")
   public static CompactionDescriptor toCompactionDescriptor(HRegionInfo info, byte[] family,
       List<Path> inputPaths, List<Path> outputPaths, Path storeDir) {
     return toCompactionDescriptor(info, null, family, inputPaths, outputPaths, storeDir);
   }
 
+  @SuppressWarnings("deprecation")
   public static CompactionDescriptor toCompactionDescriptor(HRegionInfo info, byte[] regionName,
       byte[] family, List<Path> inputPaths, List<Path> outputPaths, Path storeDir) {
     // compaction descriptor contains relative paths.
@@ -3674,28 +3663,4 @@ public final class ProtobufUtil {
     return new RegionLoadStats(stats.getMemstoreLoad(), stats.getHeapOccupancy(),
         stats.getCompactionPressure());
   }
-
-  /**
-   * @param msg
-   * @return A String version of the passed in <code>msg</code>
-   */
-  public static String toText(Message msg) {
-    return TextFormat.shortDebugString(msg);
-  }
-
-  public static byte [] toBytes(ByteString bs) {
-    return bs.toByteArray();
-  }
-
-  /**
-   * Contain ServiceException inside here. Take a callable that is doing our pb rpc and run it.
-   * @throws IOException
-   */
-  public static <T> T call(Callable<T> callable) throws IOException {
-    try {
-      return callable.call();
-    } catch (Exception e) {
-      throw ProtobufUtil.handleRemoteException(e);
-    }
-  }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0206dc67/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientScanner.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientScanner.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientScanner.java
index fd2a393..f083001 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientScanner.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientScanner.java
@@ -65,6 +65,7 @@ public class TestClientScanner {
   RpcControllerFactory controllerFactory;
 
   @Before
+  @SuppressWarnings("deprecation")
   public void setup() throws IOException {
     clusterConn = Mockito.mock(ClusterConnection.class);
     rpcFactory = Mockito.mock(RpcRetryingCallerFactory.class);

http://git-wip-us.apache.org/repos/asf/hbase/blob/0206dc67/hbase-common/src/main/java/org/apache/hadoop/hbase/HBaseIOException.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HBaseIOException.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HBaseIOException.java
index edcbdc5..9c3367e 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HBaseIOException.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HBaseIOException.java
@@ -45,5 +45,4 @@ public class HBaseIOException extends IOException {
 
   public HBaseIOException(Throwable cause) {
       super(cause);
-  }
-}
\ No newline at end of file
+  }}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0206dc67/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ExceptionUtil.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ExceptionUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ExceptionUtil.java
index 7e6c5d6..688b51a 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ExceptionUtil.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ExceptionUtil.java
@@ -73,4 +73,4 @@ public class ExceptionUtil {
 
   private ExceptionUtil() {
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0206dc67/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
index ec28315..73226aa 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
@@ -75,17 +75,20 @@ import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseIOException;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.Server;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.client.Operation;
 import org.apache.hadoop.hbase.client.VersionInfoUtil;
 import org.apache.hadoop.hbase.codec.Codec;
 import org.apache.hadoop.hbase.conf.ConfigurationObserver;
 import org.apache.hadoop.hbase.exceptions.RegionMovedException;
 import org.apache.hadoop.hbase.io.ByteBufferInputStream;
-import org.apache.hadoop.hbase.io.ByteBufferListOutputStream;
 import org.apache.hadoop.hbase.io.ByteBufferOutputStream;
 import org.apache.hadoop.hbase.io.ByteBufferPool;
+import org.apache.hadoop.hbase.io.ByteBufferListOutputStream;
 import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler;
 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
@@ -96,6 +99,7 @@ import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse;
 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader;
 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader;
 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.security.AccessDeniedException;
 import org.apache.hadoop.hbase.security.AuthMethod;
 import org.apache.hadoop.hbase.security.HBasePolicyProvider;

http://git-wip-us.apache.org/repos/asf/hbase/blob/0206dc67/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
index 8ddbe18..09dedec 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
@@ -87,8 +87,6 @@ import org.apache.hadoop.hbase.io.hfile.HFileContext;
 import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
 import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder;
 import org.apache.hadoop.hbase.io.hfile.HFileScanner;
-import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
-import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.regionserver.BloomType;
 import org.apache.hadoop.hbase.regionserver.HStore;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
@@ -104,6 +102,7 @@ import org.apache.hadoop.util.ToolRunner;
 
 /**
  * Tool to load the output of HFileOutputFormat into an existing table.
+ * @see #usage()
  */
 @InterfaceAudience.Public
 @InterfaceStability.Stable
@@ -131,13 +130,11 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
   private String bulkToken;
   private UserProvider userProvider;
   private int nrThreads;
-  private RpcControllerFactory rpcControllerFactory;
 
   private LoadIncrementalHFiles() {}
 
   public LoadIncrementalHFiles(Configuration conf) throws Exception {
     super(conf);
-    this.rpcControllerFactory = new RpcControllerFactory(conf);
     initialize();
   }
 
@@ -325,7 +322,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
     // LQI queue does not need to be threadsafe -- all operations on this queue
     // happen in this thread
     Deque<LoadQueueItem> queue = new LinkedList<>();
-    SecureBulkLoadClient secureClient =  new SecureBulkLoadClient(table.getConfiguration(), table);
+    SecureBulkLoadClient secureClient =  new SecureBulkLoadClient(table);
 
     try {
       /*
@@ -476,11 +473,9 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
 
   /**
    * Used by the replication sink to load the hfiles from the source cluster. It does the following,
-   * <ol>
-   * <li>LoadIncrementalHFiles#groupOrSplitPhase(Table, ExecutorService, Deque, Pair)}</li>
-   * <li>LoadIncrementalHFiles#bulkLoadPhase(Table, Connection, ExecutorService, Deque, Multimap)
-   * </li>
-   * </ol>
+   * 1. {@link LoadIncrementalHFiles#groupOrSplitPhase(Table, ExecutorService, Deque, Pair)} 2.
+   * {@link
+   * LoadIncrementalHFiles#bulkLoadPhase(Table, Connection, ExecutorService, Deque, Multimap)}
    * @param table Table to which these hfiles should be loaded to
    * @param conn Connection to use
    * @param queue {@link LoadQueueItem} has hfiles yet to be loaded
@@ -781,23 +776,27 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
   protected List<LoadQueueItem> tryAtomicRegionLoad(final Connection conn,
       final TableName tableName, final byte[] first, final Collection<LoadQueueItem> lqis)
   throws IOException {
-    final List<Pair<byte[], String>> famPaths = new ArrayList<>(lqis.size());
+    final List<Pair<byte[], String>> famPaths =
+      new ArrayList<>(lqis.size());
     for (LoadQueueItem lqi : lqis) {
       famPaths.add(Pair.newPair(lqi.family, lqi.hfilePath.toString()));
     }
-    final RegionServerCallable<Boolean> svrCallable = new RegionServerCallable<Boolean>(conn,
-        rpcControllerFactory, tableName, first) {
+
+    final RegionServerCallable<Boolean> svrCallable =
+        new RegionServerCallable<Boolean>(conn, tableName, first) {
       @Override
-      protected Boolean call(PayloadCarryingRpcController controller) throws Exception {
+      public Boolean call(int callTimeout) throws Exception {
         SecureBulkLoadClient secureClient = null;
         boolean success = false;
+
         try {
           LOG.debug("Going to connect to server " + getLocation() + " for row "
               + Bytes.toStringBinary(getRow()) + " with hfile group " + famPaths);
           byte[] regionName = getLocation().getRegionInfo().getRegionName();
           try (Table table = conn.getTable(getTableName())) {
-            secureClient = new SecureBulkLoadClient(getConf(), table);
-            success = secureClient.secureBulkLoadHFiles(getStub(), famPaths, regionName,
+            secureClient = new SecureBulkLoadClient(table);
+            success =
+                secureClient.secureBulkLoadHFiles(getStub(), famPaths, regionName,
                   assignSeqIds, fsDelegationToken.getUserToken(), bulkToken);
           }
           return success;
@@ -1079,7 +1078,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
 
   /**
    * Called from replication sink, where it manages bulkToken(staging directory) by itself. This is
-   * used only when SecureBulkLoadEndpoint is configured in hbase.coprocessor.region.classes
+   * used only when {@link SecureBulkLoadEndpoint} is configured in hbase.coprocessor.region.classes
    * property. This directory is used as a temporary directory where all files are initially
    * copied/moved from user given directory, set all the required file permissions and then from
    * their it is finally loaded into a table. This should be set only when, one would like to manage
@@ -1089,4 +1088,5 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
   public void setBulkToken(String stagingDir) {
     this.bulkToken = stagingDir;
   }
-}
\ No newline at end of file
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0206dc67/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ExpiredMobFileCleanerChore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ExpiredMobFileCleanerChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ExpiredMobFileCleanerChore.java
index 3261bd6..a21edcc 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ExpiredMobFileCleanerChore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ExpiredMobFileCleanerChore.java
@@ -35,6 +35,8 @@ import org.apache.hadoop.hbase.mob.ExpiredMobFileCleaner;
 import org.apache.hadoop.hbase.mob.MobConstants;
 import org.apache.hadoop.hbase.mob.MobUtils;
 
+import com.google.protobuf.ServiceException;
+
 /**
  * The Class ExpiredMobFileCleanerChore for running cleaner regularly to remove the expired
  * mob files.
@@ -84,6 +86,10 @@ public class ExpiredMobFileCleanerChore extends ScheduledChore {
             } catch (LockTimeoutException e) {
               LOG.info("Fail to acquire the lock because of timeout, maybe a"
                 + " MobCompactor is running", e);
+            } catch (ServiceException e) {
+              LOG.error(
+                "Fail to clean the expired mob files for the column " + hcd.getNameAsString()
+                  + " in the table " + htd.getNameAsString(), e);
             } catch (IOException e) {
               LOG.error(
                 "Fail to clean the expired mob files for the column " + hcd.getNameAsString()

http://git-wip-us.apache.org/repos/asf/hbase/blob/0206dc67/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
index d7ba4f3..531883a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
@@ -61,9 +61,9 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionRequest;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionResponse;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ServerInfo;
+import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
 import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds;
 import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.StoreSequenceId;
-import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
@@ -454,7 +454,8 @@ public class ServerManager {
   /**
    * Adds the onlineServers list. onlineServers should be locked.
    * @param serverName The remote servers name.
-   * @param s
+   * @param sl
+   * @return Server load from the removed server, if any.
    */
   @VisibleForTesting
   void recordNewServerWithLock(final ServerName serverName, final ServerLoad sl) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/0206dc67/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/ExpiredMobFileCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/ExpiredMobFileCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/ExpiredMobFileCleaner.java
index d4a54bb..3c965cb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/ExpiredMobFileCleaner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/ExpiredMobFileCleaner.java
@@ -41,6 +41,8 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 
+import com.google.protobuf.ServiceException;
+
 /**
  * The cleaner to delete the expired MOB files.
  */
@@ -58,8 +60,11 @@ public class ExpiredMobFileCleaner extends Configured implements Tool {
    * directory.
    * @param tableName The current table name.
    * @param family The current family.
+   * @throws ServiceException
+   * @throws IOException
    */
-  public void cleanExpiredMobFiles(String tableName, HColumnDescriptor family) throws IOException {
+  public void cleanExpiredMobFiles(String tableName, HColumnDescriptor family)
+      throws ServiceException, IOException {
     Configuration conf = getConf();
     TableName tn = TableName.valueOf(tableName);
     FileSystem fs = FileSystem.get(conf);
@@ -94,7 +99,7 @@ public class ExpiredMobFileCleaner extends Configured implements Tool {
     String tableName = args[0];
     String familyName = args[1];
     TableName tn = TableName.valueOf(tableName);
-    HBaseAdmin.available(getConf());
+    HBaseAdmin.checkHBaseAvailable(getConf());
     Connection connection = ConnectionFactory.createConnection(getConf());
     Admin admin = connection.getAdmin();
     try {
@@ -122,4 +127,5 @@ public class ExpiredMobFileCleaner extends Configured implements Tool {
       }
     }
   }
-}
\ No newline at end of file
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0206dc67/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/mapreduce/Sweeper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/mapreduce/Sweeper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/mapreduce/Sweeper.java
index c27e8ae..8547c8c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/mapreduce/Sweeper.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/mapreduce/Sweeper.java
@@ -38,6 +38,8 @@ import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.zookeeper.KeeperException;
 
+import com.google.protobuf.ServiceException;
+
 /**
  * The sweep tool. It deletes the mob files that are not used and merges the small mob files to
  * bigger ones. Each run of this sweep tool only handles one column family. The runs on
@@ -62,10 +64,10 @@ public class Sweeper extends Configured implements Tool {
    * @throws ServiceException
    */
   int sweepFamily(String tableName, String familyName) throws IOException, InterruptedException,
-      ClassNotFoundException, KeeperException {
+      ClassNotFoundException, KeeperException, ServiceException {
     Configuration conf = getConf();
     // make sure the target HBase exists.
-    HBaseAdmin.available(conf);
+    HBaseAdmin.checkHBaseAvailable(conf);
     Connection connection = ConnectionFactory.createConnection(getConf());
     Admin admin = connection.getAdmin();
     try {

http://git-wip-us.apache.org/repos/asf/hbase/blob/0206dc67/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index fb9a605..d87ada4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -87,6 +87,7 @@ import org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface;
 import org.apache.hadoop.hbase.ipc.RpcServerInterface;
 import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
 import org.apache.hadoop.hbase.ipc.ServerRpcController;
+import org.apache.hadoop.hbase.ipc.TimeLimitedRpcController;
 import org.apache.hadoop.hbase.master.MasterRpcServices;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.RequestConverter;
@@ -2764,15 +2765,13 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
                     timeLimitDelta =
                         scannerLeaseTimeoutPeriod > 0 ? scannerLeaseTimeoutPeriod : rpcTimeout;
                   }
-                  if (controller instanceof PayloadCarryingRpcController) {
-                    PayloadCarryingRpcController pRpcController =
-                        (PayloadCarryingRpcController)controller;
-                    if (pRpcController.getCallTimeout() > 0) {
-                      timeLimitDelta = Math.min(timeLimitDelta, pRpcController.getCallTimeout());
+                  if (controller instanceof TimeLimitedRpcController) {
+                    TimeLimitedRpcController timeLimitedRpcController =
+                        (TimeLimitedRpcController)controller;
+                    if (timeLimitedRpcController.getCallTimeout() > 0) {
+                      timeLimitDelta = Math.min(timeLimitDelta,
+                          timeLimitedRpcController.getCallTimeout());
                     }
-                  } else {
-                    throw new UnsupportedOperationException("We only do " +
-                      "PayloadCarryingRpcControllers! FIX IF A PROBLEM");
                   }
                   // Use half of whichever timeout value was more restrictive... But don't allow
                   // the time limit to be less than the allowable minimum (could cause an

http://git-wip-us.apache.org/repos/asf/hbase/blob/0206dc67/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEditsReplaySink.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEditsReplaySink.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEditsReplaySink.java
index c71153d..3eb85bd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEditsReplaySink.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEditsReplaySink.java
@@ -18,6 +18,8 @@
  */
 package org.apache.hadoop.hbase.regionserver.wal;
 
+import com.google.protobuf.ServiceException;
+
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
@@ -42,6 +44,7 @@ import org.apache.hadoop.hbase.client.RegionServerCallable;
 import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
 import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.ReplicationProtbufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
@@ -58,8 +61,10 @@ import org.apache.hadoop.hbase.wal.WAL.Entry;
  */
 @InterfaceAudience.Private
 public class WALEditsReplaySink {
+
   private static final Log LOG = LogFactory.getLog(WALEditsReplaySink.class);
   private static final int MAX_BATCH_SIZE = 1024;
+
   private final Configuration conf;
   private final ClusterConnection conn;
   private final TableName tableName;
@@ -161,8 +166,8 @@ public class WALEditsReplaySink {
     try {
       RpcRetryingCallerFactory factory = RpcRetryingCallerFactory.instantiate(conf, null);
       ReplayServerCallable<ReplicateWALEntryResponse> callable =
-          new ReplayServerCallable<ReplicateWALEntryResponse>(this.conn, this.rpcControllerFactory,
-              this.tableName, regionLoc, entries);
+          new ReplayServerCallable<ReplicateWALEntryResponse>(this.conn, this.tableName, regionLoc,
+              regionInfo, entries);
       factory.<ReplicateWALEntryResponse> newCaller().callWithRetries(callable, this.replayTimeout);
     } catch (IOException ie) {
       if (skipErrors) {
@@ -179,19 +184,31 @@ public class WALEditsReplaySink {
    * @param <R>
    */
   class ReplayServerCallable<R> extends RegionServerCallable<ReplicateWALEntryResponse> {
+    private HRegionInfo regionInfo;
     private List<Entry> entries;
 
-    ReplayServerCallable(final Connection connection, RpcControllerFactory rpcControllerFactory,
-        final TableName tableName, final HRegionLocation regionLoc, final List<Entry> entries) {
-      super(connection, rpcControllerFactory, tableName, null);
+    ReplayServerCallable(final Connection connection, final TableName tableName,
+        final HRegionLocation regionLoc, final HRegionInfo regionInfo,
+        final List<Entry> entries) {
+      super(connection, tableName, null);
       this.entries = entries;
+      this.regionInfo = regionInfo;
       setLocation(regionLoc);
     }
 
     @Override
-    protected ReplicateWALEntryResponse call(PayloadCarryingRpcController controller)
-    throws Exception {
-      if (entries.isEmpty()) return null;
+    public ReplicateWALEntryResponse call(int callTimeout) throws IOException {
+      try {
+        replayToServer(this.regionInfo, this.entries);
+      } catch (ServiceException se) {
+        throw ProtobufUtil.getRemoteException(se);
+      }
+      return null;
+    }
+
+    private void replayToServer(HRegionInfo regionInfo, List<Entry> entries)
+        throws IOException, ServiceException {
+      if (entries.isEmpty()) return;
 
       Entry[] entriesArray = new Entry[entries.size()];
       entriesArray = entries.toArray(entriesArray);
@@ -199,8 +216,12 @@ public class WALEditsReplaySink {
 
       Pair<AdminProtos.ReplicateWALEntryRequest, CellScanner> p =
           ReplicationProtbufUtil.buildReplicateWALEntryRequest(entriesArray);
-      controller.setCellScanner(p.getSecond());
-      return remoteSvr.replay(controller, p.getFirst());
+      PayloadCarryingRpcController controller = rpcControllerFactory.newController(p.getSecond());
+      try {
+        remoteSvr.replay(controller, p.getFirst());
+      } catch (ServiceException se) {
+        throw ProtobufUtil.getRemoteException(se);
+      }
     }
 
     @Override
@@ -224,4 +245,4 @@ public class WALEditsReplaySink {
       }
     }
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0206dc67/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java
index c756294..b0fd176 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java
@@ -34,6 +34,7 @@ import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.CellScanner;
@@ -45,21 +46,27 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.RegionLocations;
-import org.apache.hadoop.hbase.TableDescriptors;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.TableDescriptors;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.RegionAdminServiceCallable;
+import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
 import org.apache.hadoop.hbase.client.RetryingCallable;
 import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
 import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.ReplicationProtbufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryResponse;
+import org.apache.hadoop.hbase.wal.WAL.Entry;
+import org.apache.hadoop.hbase.wal.WALSplitter.EntryBuffers;
+import org.apache.hadoop.hbase.wal.WALSplitter.OutputSink;
+import org.apache.hadoop.hbase.wal.WALSplitter.PipelineController;
+import org.apache.hadoop.hbase.wal.WALSplitter.RegionEntryBuffer;
+import org.apache.hadoop.hbase.wal.WALSplitter.SinkWriter;
 import org.apache.hadoop.hbase.replication.BaseWALEntryFilter;
 import org.apache.hadoop.hbase.replication.ChainWALEntryFilter;
 import org.apache.hadoop.hbase.replication.HBaseReplicationEndpoint;
@@ -67,17 +74,12 @@ import org.apache.hadoop.hbase.replication.WALEntryFilter;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.Threads;
-import org.apache.hadoop.hbase.wal.WAL.Entry;
-import org.apache.hadoop.hbase.wal.WALSplitter.EntryBuffers;
-import org.apache.hadoop.hbase.wal.WALSplitter.OutputSink;
-import org.apache.hadoop.hbase.wal.WALSplitter.PipelineController;
-import org.apache.hadoop.hbase.wal.WALSplitter.RegionEntryBuffer;
-import org.apache.hadoop.hbase.wal.WALSplitter.SinkWriter;
 import org.apache.hadoop.util.StringUtils;
 
 import com.google.common.cache.Cache;
 import com.google.common.cache.CacheBuilder;
 import com.google.common.collect.Lists;
+import com.google.protobuf.ServiceException;
 
 /**
  * A {@link org.apache.hadoop.hbase.replication.ReplicationEndpoint} endpoint
@@ -609,8 +611,9 @@ public class RegionReplicaReplicationEndpoint extends HBaseReplicationEndpoint {
    * Calls replay on the passed edits for the given set of entries belonging to the region. It skips
    * the entry if the region boundaries have changed or the region is gone.
    */
-  static class RegionReplicaReplayCallable extends
-      RegionAdminServiceCallable<ReplicateWALEntryResponse> {
+  static class RegionReplicaReplayCallable
+    extends RegionAdminServiceCallable<ReplicateWALEntryResponse> {
+
     private final List<Entry> entries;
     private final byte[] initialEncodedRegionName;
     private final AtomicLong skippedEntries;
@@ -625,25 +628,38 @@ public class RegionReplicaReplicationEndpoint extends HBaseReplicationEndpoint {
       this.initialEncodedRegionName = regionInfo.getEncodedNameAsBytes();
     }
 
-    public ReplicateWALEntryResponse call(PayloadCarryingRpcController controller) throws Exception {
-      // Check whether we should still replay this entry. If the regions are changed, or the
+    @Override
+    public ReplicateWALEntryResponse call(int timeout) throws IOException {
+      return replayToServer(this.entries, timeout);
+    }
+
+    private ReplicateWALEntryResponse replayToServer(List<Entry> entries, int timeout)
+        throws IOException {
+      // check whether we should still replay this entry. If the regions are changed, or the
       // entry is not coming form the primary region, filter it out because we do not need it.
       // Regions can change because of (1) region split (2) region merge (3) table recreated
       boolean skip = false;
+
       if (!Bytes.equals(location.getRegionInfo().getEncodedNameAsBytes(),
-          initialEncodedRegionName)) {
+        initialEncodedRegionName)) {
         skip = true;
       }
-      if (!this.entries.isEmpty() && !skip) {
-        Entry[] entriesArray = new Entry[this.entries.size()];
-        entriesArray = this.entries.toArray(entriesArray);
+      if (!entries.isEmpty() && !skip) {
+        Entry[] entriesArray = new Entry[entries.size()];
+        entriesArray = entries.toArray(entriesArray);
 
         // set the region name for the target region replica
         Pair<AdminProtos.ReplicateWALEntryRequest, CellScanner> p =
             ReplicationProtbufUtil.buildReplicateWALEntryRequest(entriesArray, location
                 .getRegionInfo().getEncodedNameAsBytes(), null, null, null);
-        controller.setCellScanner(p.getSecond());
-        return stub.replay(controller, p.getFirst());
+        try {
+          PayloadCarryingRpcController controller = rpcControllerFactory.newController(p.getSecond());
+          controller.setCallTimeout(timeout);
+          controller.setPriority(tableName);
+          return stub.replay(controller, p.getFirst());
+        } catch (ServiceException se) {
+          throw ProtobufUtil.getRemoteException(se);
+        }
       }
 
       if (skip) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/0206dc67/hbase-server/src/main/java/org/apache/hadoop/hbase/util/Merge.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/Merge.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/Merge.java
index 3c81cfe..d708edc 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/Merge.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/Merge.java
@@ -23,18 +23,19 @@ import java.io.IOException;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.MetaTableAccessor;
-import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.MasterNotRunningException;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
@@ -79,11 +80,13 @@ public class Merge extends Configured implements Tool {
     // Verify HBase is down
     LOG.info("Verifying that HBase is not running...");
     try {
-      HBaseAdmin.available(getConf());
+      HBaseAdmin.checkHBaseAvailable(getConf());
       LOG.fatal("HBase cluster must be off-line, and is not. Aborting.");
       return -1;
     } catch (ZooKeeperConnectionException zkce) {
       // If no zk, presume no master.
+    } catch (MasterNotRunningException e) {
+      // Expected. Ignore.
     }
 
     // Initialize MetaUtils and and get the root of the HBase installation

http://git-wip-us.apache.org/repos/asf/hbase/blob/0206dc67/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java
index 2dca6b1..d778fa9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java
@@ -60,6 +60,7 @@ public class TestNamespace {
   private static ZKNamespaceManager zkNamespaceManager;
   private String prefix = "TestNamespace";
 
+
   @BeforeClass
   public static void setUp() throws Exception {
     TEST_UTIL = new HBaseTestingUtility();
@@ -300,8 +301,7 @@ public class TestNamespace {
     runWithExpectedException(new Callable<Void>() {
       @Override
       public Void call() throws Exception {
-        HTableDescriptor htd =
-            new HTableDescriptor(TableName.valueOf("non_existing_namespace", "table1"));
+        HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("non_existing_namespace", "table1"));
         htd.addFamily(new HColumnDescriptor("family1"));
         admin.createTable(htd);
         return null;
@@ -387,4 +387,5 @@ public class TestNamespace {
     }
     fail("Should have thrown exception " + exceptionClass);
   }
-}
\ No newline at end of file
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0206dc67/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
index 1716622..d088fc4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.MasterNotRunningException;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.NotServingRegionException;
 import org.apache.hadoop.hbase.ProcedureInfo;
@@ -58,6 +59,7 @@ import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
+import org.apache.hadoop.hbase.wal.FSHLogProvider;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -65,6 +67,8 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import com.google.protobuf.ServiceException;
+
 
 /**
  * Class to test HBaseAdmin.
@@ -639,9 +643,11 @@ public class TestAdmin2 {
 
     long start = System.currentTimeMillis();
     try {
-      HBaseAdmin.available(conf);
+      HBaseAdmin.checkHBaseAvailable(conf);
       assertTrue(false);
+    } catch (MasterNotRunningException ignored) {
     } catch (ZooKeeperConnectionException ignored) {
+    } catch (ServiceException ignored) {
     } catch (IOException ignored) {
     }
     long end = System.currentTimeMillis();

http://git-wip-us.apache.org/repos/asf/hbase/blob/0206dc67/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java
index f49c558..679d9c9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java
@@ -28,10 +28,13 @@ import java.net.UnknownHostException;
 import java.util.Random;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.MasterNotRunningException;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
 import org.apache.hadoop.hbase.ipc.AbstractRpcClient;
@@ -53,6 +56,7 @@ import com.google.protobuf.ServiceException;
 
 @Category({MediumTests.class, ClientTests.class})
 public class TestClientTimeouts {
+  private static final Log LOG = LogFactory.getLog(TestClientTimeouts.class);
   private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
   protected static int SLAVES = 1;
 
@@ -83,6 +87,7 @@ public class TestClientTimeouts {
    */
   @Test
   public void testAdminTimeout() throws Exception {
+    Connection lastConnection = null;
     boolean lastFailed = false;
     int initialInvocations = RandomTimeoutBlockingRpcChannel.invokations.get();
     RandomTimeoutRpcClient rpcClient = (RandomTimeoutRpcClient) RpcClientFactory
@@ -100,7 +105,7 @@ public class TestClientTimeouts {
           connection = ConnectionFactory.createConnection(conf);
           admin = connection.getAdmin();
           // run some admin commands
-          HBaseAdmin.available(conf);
+          HBaseAdmin.checkHBaseAvailable(conf);
           admin.setBalancerRunning(false, false);
         } catch (ZooKeeperConnectionException ex) {
           // Since we are randomly throwing SocketTimeoutExceptions, it is possible to get

http://git-wip-us.apache.org/repos/asf/hbase/blob/0206dc67/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 33af5de..1b20b76 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
@@ -61,7 +61,6 @@ import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.exceptions.RegionMovedException;
 import org.apache.hadoop.hbase.filter.Filter;
 import org.apache.hadoop.hbase.filter.FilterBase;
-import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
 import org.apache.hadoop.hbase.ipc.RpcClient;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.master.HMaster;
@@ -104,6 +103,8 @@ public class TestHCM {
       TableName.valueOf("test2");
   private static final TableName TABLE_NAME3 =
       TableName.valueOf("test3");
+  private static final TableName TABLE_NAME4 =
+      TableName.valueOf("test4");
   private static final byte[] FAM_NAM = Bytes.toBytes("f");
   private static final byte[] ROW = Bytes.toBytes("bbb");
   private static final byte[] ROW_X = Bytes.toBytes("xxx");
@@ -406,11 +407,10 @@ public class TestHCM {
     long pauseTime;
     long baseTime = 100;
     TableName tableName = TableName.valueOf("HCM-testCallableSleep");
+    Table table = TEST_UTIL.createTable(tableName, FAM_NAM);
     RegionServerCallable<Object> regionServerCallable = new RegionServerCallable<Object>(
-        TEST_UTIL.getConnection(), new RpcControllerFactory(TEST_UTIL.getConfiguration()),
-        tableName, ROW) {
-      @Override
-      protected Object call(PayloadCarryingRpcController controller) throws Exception {
+        TEST_UTIL.getConnection(), tableName, ROW) {
+      public Object call(int timeout) throws IOException {
         return null;
       }
     };
@@ -424,10 +424,9 @@ public class TestHCM {
 
     RegionAdminServiceCallable<Object> regionAdminServiceCallable =
         new RegionAdminServiceCallable<Object>(
-        (ClusterConnection) TEST_UTIL.getConnection(),
-          new RpcControllerFactory(TEST_UTIL.getConfiguration()), tableName, ROW) {
-      @Override
-      public Object call(PayloadCarryingRpcController controller) throws Exception {
+        (ClusterConnection) TEST_UTIL.getConnection(), new RpcControllerFactory(
+            TEST_UTIL.getConfiguration()), tableName, ROW) {
+      public Object call(int timeout) throws IOException {
         return null;
       }
     };
@@ -439,21 +438,16 @@ public class TestHCM {
       assertTrue(pauseTime <= (baseTime * HConstants.RETRY_BACKOFF[i] * 1.01f));
     }
 
-    MasterCallable<Object> masterCallable = new MasterCallable<Object>(TEST_UTIL.getConnection(),
-        new RpcControllerFactory(TEST_UTIL.getConfiguration())) {
-      @Override
-      protected Object call(PayloadCarryingRpcController rpcController) throws Exception {
+    MasterCallable masterCallable = new MasterCallable(TEST_UTIL.getConnection()) {
+      public Object call(int timeout) throws IOException {
         return null;
       }
     };
-    try {
-      for (int i = 0; i < HConstants.RETRY_BACKOFF.length; i++) {
-        pauseTime = masterCallable.sleep(baseTime, i);
-        assertTrue(pauseTime >= (baseTime * HConstants.RETRY_BACKOFF[i]));
-        assertTrue(pauseTime <= (baseTime * HConstants.RETRY_BACKOFF[i] * 1.01f));
-      }
-    } finally {
-      masterCallable.close();
+
+    for (int i = 0; i < HConstants.RETRY_BACKOFF.length; i++) {
+      pauseTime = masterCallable.sleep(baseTime, i);
+      assertTrue(pauseTime >= (baseTime * HConstants.RETRY_BACKOFF[i]));
+      assertTrue(pauseTime <= (baseTime * HConstants.RETRY_BACKOFF[i] * 1.01f));
     }
   }
 
@@ -1155,6 +1149,7 @@ public class TestHCM {
     ManualEnvironmentEdge timeMachine = new ManualEnvironmentEdge();
     EnvironmentEdgeManager.injectEdge(timeMachine);
     try {
+      long timeBase = timeMachine.currentTime();
       long largeAmountOfTime = ANY_PAUSE * 1000;
       ConnectionImplementation.ServerErrorTracker tracker =
           new ConnectionImplementation.ServerErrorTracker(largeAmountOfTime, 100);

http://git-wip-us.apache.org/repos/asf/hbase/blob/0206dc67/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
index d99d2ee..354f0a8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
@@ -35,8 +35,6 @@ import org.apache.hadoop.hbase.client.replication.ReplicationAdmin;
 import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
-import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.regionserver.StorefileRefresherChore;
 import org.apache.hadoop.hbase.regionserver.TestHRegionServerBulkLoad;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
@@ -334,27 +332,26 @@ public class TestReplicaWithCluster {
 
     // bulk load HFiles
     LOG.debug("Loading test data");
+    @SuppressWarnings("deprecation")
     final ClusterConnection conn = (ClusterConnection) HTU.getAdmin().getConnection();
     table = conn.getTable(hdt.getTableName());
-    final String bulkToken =
-        new SecureBulkLoadClient(HTU.getConfiguration(), table).prepareBulkLoad(conn);
-    RegionServerCallable<Void> callable = new RegionServerCallable<Void>(conn,
-        new RpcControllerFactory(HTU.getConfiguration()), hdt.getTableName(),
-        TestHRegionServerBulkLoad.rowkey(0)) {
-      @Override
-      protected Void call(PayloadCarryingRpcController controller) throws Exception {
-        LOG.debug("Going to connect to server " + getLocation() + " for row "
+    final String bulkToken = new SecureBulkLoadClient(table).prepareBulkLoad(conn);
+    RegionServerCallable<Void> callable = new RegionServerCallable<Void>(
+      conn, hdt.getTableName(), TestHRegionServerBulkLoad.rowkey(0)) {
+        @Override
+        public Void call(int timeout) throws Exception {
+          LOG.debug("Going to connect to server " + getLocation() + " for row "
             + Bytes.toStringBinary(getRow()));
-        SecureBulkLoadClient secureClient = null;
-        byte[] regionName = getLocation().getRegionInfo().getRegionName();
-        try (Table table = conn.getTable(getTableName())) {
-          secureClient = new SecureBulkLoadClient(HTU.getConfiguration(), table);
-          secureClient.secureBulkLoadHFiles(getStub(), famPaths, regionName,
-              true, null, bulkToken);
+          SecureBulkLoadClient secureClient = null;
+          byte[] regionName = getLocation().getRegionInfo().getRegionName();
+          try (Table table = conn.getTable(getTableName())) {
+            secureClient = new SecureBulkLoadClient(table);
+            secureClient.secureBulkLoadHFiles(getStub(), famPaths, regionName,
+                  true, null, bulkToken);
+          }
+          return null;
         }
-        return null;
-      }
-    };
+      };
     RpcRetryingCallerFactory factory = new RpcRetryingCallerFactory(HTU.getConfiguration());
     RpcRetryingCaller<Void> caller = factory.newCaller();
     caller.callWithRetries(callable, 10000);

http://git-wip-us.apache.org/repos/asf/hbase/blob/0206dc67/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java
index 30805c0..6e68201 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java
@@ -62,8 +62,6 @@ import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.io.hfile.HFileContext;
 import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
-import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
-import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionRequest;
@@ -200,20 +198,19 @@ public class TestHRegionServerBulkLoad {
       }
 
       // bulk load HFiles
-      final ClusterConnection conn = (ClusterConnection)UTIL.getConnection();
+      final ClusterConnection conn = (ClusterConnection) UTIL.getAdmin().getConnection();
       Table table = conn.getTable(tableName);
-      final String bulkToken = new SecureBulkLoadClient(UTIL.getConfiguration(), table).
-          prepareBulkLoad(conn);
-      RegionServerCallable<Void> callable = new RegionServerCallable<Void>(conn,
-          new RpcControllerFactory(UTIL.getConfiguration()), tableName, Bytes.toBytes("aaa")) {
+      final String bulkToken = new SecureBulkLoadClient(table).prepareBulkLoad(conn);
+      RegionServerCallable<Void> callable =
+          new RegionServerCallable<Void>(conn, tableName, Bytes.toBytes("aaa")) {
         @Override
-        public Void call(PayloadCarryingRpcController controller) throws Exception {
+        public Void call(int callTimeout) throws Exception {
           LOG.debug("Going to connect to server " + getLocation() + " for row "
               + Bytes.toStringBinary(getRow()));
           SecureBulkLoadClient secureClient = null;
           byte[] regionName = getLocation().getRegionInfo().getRegionName();
           try (Table table = conn.getTable(getTableName())) {
-            secureClient = new SecureBulkLoadClient(UTIL.getConfiguration(), table);
+            secureClient = new SecureBulkLoadClient(table);
             secureClient.secureBulkLoadHFiles(getStub(), famPaths, regionName,
                   true, null, bulkToken);
           }
@@ -227,15 +224,15 @@ public class TestHRegionServerBulkLoad {
       // Periodically do compaction to reduce the number of open file handles.
       if (numBulkLoads.get() % 5 == 0) {
         // 5 * 50 = 250 open file handles!
-        callable = new RegionServerCallable<Void>(conn,
-            new RpcControllerFactory(UTIL.getConfiguration()), tableName, Bytes.toBytes("aaa")) {
+        callable = new RegionServerCallable<Void>(conn, tableName, Bytes.toBytes("aaa")) {
           @Override
-          protected Void call(PayloadCarryingRpcController controller) throws Exception {
+          public Void call(int callTimeout) throws Exception {
             LOG.debug("compacting " + getLocation() + " for row "
                 + Bytes.toStringBinary(getRow()));
             AdminProtos.AdminService.BlockingInterface server =
               conn.getAdmin(getLocation().getServerName());
-            CompactRegionRequest request = RequestConverter.buildCompactRegionRequest(
+            CompactRegionRequest request =
+              RequestConverter.buildCompactRegionRequest(
                 getLocation().getRegionInfo().getRegionName(), true, null);
             server.compactRegion(null, request);
             numCompactions.incrementAndGet();

http://git-wip-us.apache.org/repos/asf/hbase/blob/0206dc67/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoadWithOldClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoadWithOldClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoadWithOldClient.java
index 7560a41..d55adef 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoadWithOldClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoadWithOldClient.java
@@ -33,8 +33,6 @@ import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.RegionServerCallable;
 import org.apache.hadoop.hbase.client.RpcRetryingCaller;
 import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
-import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
-import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionRequest;
@@ -91,12 +89,10 @@ public class TestHRegionServerBulkLoadWithOldClient extends TestHRegionServerBul
 
       // bulk load HFiles
       final ClusterConnection conn = (ClusterConnection) UTIL.getAdmin().getConnection();
-      RpcControllerFactory rpcControllerFactory = new RpcControllerFactory(UTIL.getConfiguration());
       RegionServerCallable<Void> callable =
-          new RegionServerCallable<Void>(conn, rpcControllerFactory, tableName,
-              Bytes.toBytes("aaa")) {
+          new RegionServerCallable<Void>(conn, tableName, Bytes.toBytes("aaa")) {
         @Override
-        protected Void call(PayloadCarryingRpcController controller) throws Exception {
+        public Void call(int callTimeout) throws Exception {
           LOG.info("Non-secure old client");
           byte[] regionName = getLocation().getRegionInfo().getRegionName();
               BulkLoadHFileRequest request =
@@ -113,10 +109,9 @@ public class TestHRegionServerBulkLoadWithOldClient extends TestHRegionServerBul
       // Periodically do compaction to reduce the number of open file handles.
       if (numBulkLoads.get() % 5 == 0) {
         // 5 * 50 = 250 open file handles!
-        callable = new RegionServerCallable<Void>(conn, rpcControllerFactory, tableName,
-            Bytes.toBytes("aaa")) {
+        callable = new RegionServerCallable<Void>(conn, tableName, Bytes.toBytes("aaa")) {
           @Override
-          protected Void call(PayloadCarryingRpcController controller) throws Exception {
+          public Void call(int callTimeout) throws Exception {
             LOG.debug("compacting " + getLocation() + " for row "
                 + Bytes.toStringBinary(getRow()));
             AdminProtos.AdminService.BlockingInterface server =

http://git-wip-us.apache.org/repos/asf/hbase/blob/0206dc67/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoadWithOldSecureEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoadWithOldSecureEndpoint.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoadWithOldSecureEndpoint.java
index 0bc9498..6de6261 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoadWithOldSecureEndpoint.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoadWithOldSecureEndpoint.java
@@ -33,13 +33,13 @@ import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.RegionServerCallable;
 import org.apache.hadoop.hbase.client.RpcRetryingCaller;
 import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
+import org.apache.hadoop.hbase.client.SecureBulkLoadClient;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
-import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
-import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionRequest;
+import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequest;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -62,8 +62,7 @@ public class TestHRegionServerBulkLoadWithOldSecureEndpoint extends TestHRegionS
     super(duration);
   }
 
-  private static final Log LOG =
-      LogFactory.getLog(TestHRegionServerBulkLoadWithOldSecureEndpoint.class);
+  private static final Log LOG = LogFactory.getLog(TestHRegionServerBulkLoadWithOldSecureEndpoint.class);
 
   @BeforeClass
   public static void setUpBeforeClass() throws IOException {
@@ -104,17 +103,16 @@ public class TestHRegionServerBulkLoadWithOldSecureEndpoint extends TestHRegionS
       final ClusterConnection conn = (ClusterConnection) UTIL.getAdmin().getConnection();
       Table table = conn.getTable(tableName);
       final String bulkToken = new SecureBulkLoadEndpointClient(table).prepareBulkLoad(tableName);
-      RpcControllerFactory rpcControllerFactory = new RpcControllerFactory(UTIL.getConfiguration());
       RegionServerCallable<Void> callable =
-          new RegionServerCallable<Void>(conn, rpcControllerFactory, tableName,
-              Bytes.toBytes("aaa")) {
+          new RegionServerCallable<Void>(conn, tableName, Bytes.toBytes("aaa")) {
             @Override
-            protected Void call(PayloadCarryingRpcController controller) throws Exception {
-              LOG.debug("Going to connect to server " + getLocation() + " for row " +
-                  Bytes.toStringBinary(getRow()));
+            public Void call(int callTimeout) throws Exception {
+              LOG.debug("Going to connect to server " + getLocation() + " for row "
+                  + Bytes.toStringBinary(getRow()));
               try (Table table = conn.getTable(getTableName())) {
-                boolean loaded = new SecureBulkLoadEndpointClient(table).bulkLoadHFiles(famPaths,
-                    null, bulkToken, getLocation().getRegionInfo().getStartKey());
+                boolean loaded =
+                    new SecureBulkLoadEndpointClient(table).bulkLoadHFiles(famPaths, null,
+                      bulkToken, getLocation().getRegionInfo().getStartKey());
               }
               return null;
             }
@@ -126,10 +124,9 @@ public class TestHRegionServerBulkLoadWithOldSecureEndpoint extends TestHRegionS
       // Periodically do compaction to reduce the number of open file handles.
       if (numBulkLoads.get() % 5 == 0) {
         // 5 * 50 = 250 open file handles!
-        callable = new RegionServerCallable<Void>(conn, rpcControllerFactory, tableName,
-            Bytes.toBytes("aaa")) {
+        callable = new RegionServerCallable<Void>(conn, tableName, Bytes.toBytes("aaa")) {
           @Override
-          protected Void call(PayloadCarryingRpcController controller) throws Exception {
+          public Void call(int callTimeout) throws Exception {
             LOG.debug("compacting " + getLocation() + " for row "
                 + Bytes.toStringBinary(getRow()));
             AdminProtos.AdminService.BlockingInterface server =

http://git-wip-us.apache.org/repos/asf/hbase/blob/0206dc67/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/SparkSQLPushDownFilter.java
----------------------------------------------------------------------
diff --git a/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/SparkSQLPushDownFilter.java b/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/SparkSQLPushDownFilter.java
index 3e90fe1..fa66d69 100644
--- a/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/SparkSQLPushDownFilter.java
+++ b/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/SparkSQLPushDownFilter.java
@@ -17,6 +17,8 @@
 
 package org.apache.hadoop.hbase.spark;
 
+import com.google.protobuf.ByteString;
+import com.google.protobuf.InvalidProtocolBufferException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.Cell;
@@ -35,8 +37,6 @@ import java.io.IOException;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import com.google.protobuf.InvalidProtocolBufferException;
-import com.google.protobuf.ByteString;
 
 /**
  * This filter will push down all qualifier logic given to us


[46/50] [abbrv] hbase git commit: HBASE-15554 - StoreFile$Writer.appendGeneralBloomFilter generates extra KV (Ram)

Posted by sy...@apache.org.
HBASE-15554 - StoreFile$Writer.appendGeneralBloomFilter generates extra KV
(Ram)


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

Branch: refs/heads/hbase-12439
Commit: 285e104176bf848dc81657ac5358ad3beb49cc57
Parents: fb668a0
Author: Ramkrishna <ra...@intel.com>
Authored: Fri Aug 12 10:21:41 2016 +0530
Committer: Ramkrishna <ra...@intel.com>
Committed: Fri Aug 12 10:21:41 2016 +0530

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/CellUtil.java  |  30 ++++-
 .../hadoop/hbase/util/ByteArrayHashKey.java     |  33 +++++
 .../hadoop/hbase/util/ByteBufferUtils.java      |  16 +++
 .../apache/hadoop/hbase/util/CellHashKey.java   |  37 ++++++
 .../java/org/apache/hadoop/hbase/util/Hash.java |  17 ++-
 .../org/apache/hadoop/hbase/util/HashKey.java   |  36 ++++++
 .../apache/hadoop/hbase/util/JenkinsHash.java   | 102 +++++++--------
 .../apache/hadoop/hbase/util/MurmurHash.java    |  18 +--
 .../apache/hadoop/hbase/util/MurmurHash3.java   |  14 +-
 .../hadoop/hbase/util/RowBloomHashKey.java      |  35 +++++
 .../hadoop/hbase/util/RowColBloomHashKey.java   |  80 ++++++++++++
 .../hbase/io/hfile/CompoundBloomFilter.java     |   2 +
 .../io/hfile/CompoundBloomFilterWriter.java     |  70 +++++-----
 .../hbase/regionserver/StoreFileWriter.java     | 129 +++++--------------
 .../apache/hadoop/hbase/util/BloomContext.java  |  71 ++++++++++
 .../hadoop/hbase/util/BloomFilterChunk.java     |  45 ++++++-
 .../hadoop/hbase/util/BloomFilterFactory.java   |   4 +-
 .../hadoop/hbase/util/BloomFilterUtil.java      |  15 ++-
 .../hadoop/hbase/util/BloomFilterWriter.java    |   9 +-
 .../hadoop/hbase/util/RowBloomContext.java      |  52 ++++++++
 .../hadoop/hbase/util/RowColBloomContext.java   |  56 ++++++++
 21 files changed, 656 insertions(+), 215 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/285e1041/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
index b769f19..c6a0a93 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
@@ -26,6 +26,7 @@ import java.io.IOException;
 import java.math.BigDecimal;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map.Entry;
@@ -33,6 +34,7 @@ import java.util.NavigableMap;
 
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceAudience.Private;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.io.HeapSize;
 import org.apache.hadoop.hbase.io.TagCompressionContext;
@@ -139,6 +141,22 @@ public final class CellUtil {
     return destinationOffset + rowLen;
   }
 
+  /**
+   * Copies the row to a new byte[]
+   * @param cell the cell from which row has to copied
+   * @return the byte[] containing the row
+   */
+  public static byte[] copyRow(Cell cell) {
+    if (cell instanceof ByteBufferedCell) {
+      return ByteBufferUtils.copyOfRange(((ByteBufferedCell) cell).getRowByteBuffer(),
+        ((ByteBufferedCell) cell).getRowPosition(),
+        ((ByteBufferedCell) cell).getRowPosition() + cell.getRowLength());
+    } else {
+      return Arrays.copyOfRange(cell.getRowArray(), cell.getRowOffset(),
+        cell.getRowOffset() + cell.getRowLength());
+    }
+  }
+
   public static int copyFamilyTo(Cell cell, byte[] destination, int destinationOffset) {
     byte fLen = cell.getFamilyLength();
     if (cell instanceof ByteBufferedCell) {
@@ -200,6 +218,7 @@ public final class CellUtil {
 
   /********************* misc *************************************/
 
+  @Private
   public static byte getRowByte(Cell cell, int index) {
     if (cell instanceof ByteBufferedCell) {
       return ((ByteBufferedCell) cell).getRowByteBuffer().get(
@@ -208,6 +227,15 @@ public final class CellUtil {
     return cell.getRowArray()[cell.getRowOffset() + index];
   }
 
+  @Private
+  public static byte getQualifierByte(Cell cell, int index) {
+    if (cell instanceof ByteBufferedCell) {
+      return ((ByteBufferedCell) cell).getQualifierByteBuffer().get(
+          ((ByteBufferedCell) cell).getQualifierPosition() + index);
+    }
+    return cell.getQualifierArray()[cell.getQualifierOffset() + index];
+  }
+
   public static ByteBuffer getValueBufferShallowCopy(Cell cell) {
     ByteBuffer buffer = ByteBuffer.wrap(cell.getValueArray(), cell.getValueOffset(),
       cell.getValueLength());
@@ -1701,7 +1729,7 @@ public final class CellUtil {
 
   /**
    * Create a Cell that is smaller than all other possible Cells for the given Cell's row.
-   *
+   * The family length is considered to be 0
    * @param cell
    * @return First possible Cell on passed Cell's row.
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/285e1041/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteArrayHashKey.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteArrayHashKey.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteArrayHashKey.java
new file mode 100644
index 0000000..1c6e369
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteArrayHashKey.java
@@ -0,0 +1,33 @@
+/**
+ * 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
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.util;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+@InterfaceAudience.Private
+public class ByteArrayHashKey extends HashKey<byte[]> {
+
+  public ByteArrayHashKey(byte[] t) {
+    super(t);
+  }
+
+  @Override
+  public byte get(int pos) {
+    return t[pos];
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/285e1041/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java
index 9909f19..d788c70 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java
@@ -24,6 +24,7 @@ import java.io.OutputStream;
 import java.math.BigDecimal;
 import java.math.BigInteger;
 import java.nio.ByteBuffer;
+import java.util.Arrays;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
@@ -984,6 +985,21 @@ public final class ByteBufferUtils {
     }
   }
 
+  /**
+   * Similar to  {@link Arrays#copyOfRange(byte[], int, int)}
+   * @param original the buffer from which the copy has to happen
+   * @param from the starting index
+   * @param to the ending index
+   * @return a byte[] created out of the copy
+   */
+  public static byte[] copyOfRange(ByteBuffer original, int from, int to) {
+    int newLength = to - from;
+    if (newLength < 0) throw new IllegalArgumentException(from + " > " + to);
+    byte[] copy = new byte[newLength];
+    ByteBufferUtils.copyFromBufferToArray(copy, original, from, 0, newLength);
+    return copy;
+  }
+
   // For testing purpose
   public static String toStringBinary(final ByteBuffer b, int off, int len) {
     StringBuilder result = new StringBuilder();

http://git-wip-us.apache.org/repos/asf/hbase/blob/285e1041/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CellHashKey.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CellHashKey.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CellHashKey.java
new file mode 100644
index 0000000..3c30cc6
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CellHashKey.java
@@ -0,0 +1,37 @@
+/**
+ * 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
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.util;
+
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * Extracts the byte for the hash calculation from the given cell
+ */
+@InterfaceAudience.Private
+public abstract class CellHashKey extends HashKey<Cell> {
+
+  protected static final byte[] LATEST_TS = Bytes.toBytes(HConstants.LATEST_TIMESTAMP);
+  protected static final byte MAX_TYPE = KeyValue.Type.Maximum.getCode();
+
+  public CellHashKey(Cell cell) {
+    super(cell);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/285e1041/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Hash.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Hash.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Hash.java
index aa0795d..dafda2a 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Hash.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Hash.java
@@ -139,7 +139,20 @@ public abstract class Hash {
    * @param initval seed value
    * @return hash value
    */
-  public abstract int hash(byte[] bytes, int offset, int length, int initval);
+  // TODO : remove this once read path is updated to work with Cells
+  public int hash(byte[] bytes, int offset, int length, int initval) {
+    return hash(new ByteArrayHashKey(bytes), offset, length, initval);
+  }
 
-  // TODO : a buffer based hash function would be needed.. Not adding it for now
+  /**
+   * Calculate a hash using bytes from <code>offset</code> to <code>offset +
+   * length</code>, and the provided seed value.
+   * @param hashKey key to extract the hash
+   * @param offset offset to be used  by the hash algo
+   * @param length length to be used by the hash algo
+   * @param initval the seed value
+   * @return hash value
+   */
+  // TODO : Remove offset and length when the byte[] version of hash() is removed
+  public abstract int hash(HashKey hashKey, int offset, int length, int initval);
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/285e1041/hbase-common/src/main/java/org/apache/hadoop/hbase/util/HashKey.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/HashKey.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/HashKey.java
new file mode 100644
index 0000000..83bba3c
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/HashKey.java
@@ -0,0 +1,36 @@
+/**
+ * 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
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.util;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * Used to calculate the hash {@link Hash} algorithms for Bloomfilters.
+ *
+ * @param <T> the type of HashKey
+ */
+@InterfaceAudience.Private
+public abstract class HashKey<T> {
+  protected final T t;
+
+  public HashKey(T t) {
+    this.t = t;
+  }
+
+  public abstract byte get(int pos);
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/285e1041/hbase-common/src/main/java/org/apache/hadoop/hbase/util/JenkinsHash.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/JenkinsHash.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/JenkinsHash.java
index 789bd8d..e874bcf 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/JenkinsHash.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/JenkinsHash.java
@@ -54,9 +54,33 @@ public class JenkinsHash extends Hash {
   }
 
   /**
+   * Compute the hash of the specified file
+   * @param args name of file to compute hash of.
+   * @throws IOException e
+   */
+  public static void main(String[] args) throws IOException {
+    if (args.length != 1) {
+      System.err.println("Usage: JenkinsHash filename");
+      System.exit(-1);
+    }
+    FileInputStream in = new FileInputStream(args[0]);
+    byte[] bytes = new byte[512];
+    int value = 0;
+    JenkinsHash hash = new JenkinsHash();
+    try {
+      for (int length = in.read(bytes); length > 0; length = in.read(bytes)) {
+        value = hash.hash(bytes, length, value);
+      }
+    } finally {
+      in.close();
+    }
+    System.out.println(Math.abs(value));
+  }
+
+  /**
    * taken from  hashlittle() -- hash a variable-length key into a 32-bit value
    *
-   * @param key the key (the unaligned variable-length array of bytes)
+   * @param hashKey the key to extract the  bytes for hash algo
    * @param nbytes number of bytes to include in hash
    * @param initval can be any integer value
    * @return a 32-bit value.  Every bit of the key affects every bit of the
@@ -78,26 +102,26 @@ public class JenkinsHash extends Hash {
    * <p>Use for hash table lookup, or anything where one collision in 2^^32 is
    * acceptable.  Do NOT use for cryptographic purposes.
   */
-  @Override
   @SuppressWarnings("fallthrough")
-  public int hash(byte[] key, int off, int nbytes, int initval) {
+  @Override
+  public int hash(HashKey hashKey, int off, int nbytes, int initval) {
     int length = nbytes;
     int a, b, c;
     a = b = c = 0xdeadbeef + length + initval;
     int offset = off;
     for (; length > 12; offset += 12, length -= 12) {
-      a += (key[offset] & BYTE_MASK);
-      a += ((key[offset + 1] & BYTE_MASK) <<  8);
-      a += ((key[offset + 2] & BYTE_MASK) << 16);
-      a += ((key[offset + 3] & BYTE_MASK) << 24);
-      b += (key[offset + 4] & BYTE_MASK);
-      b += ((key[offset + 5] & BYTE_MASK) <<  8);
-      b += ((key[offset + 6] & BYTE_MASK) << 16);
-      b += ((key[offset + 7] & BYTE_MASK) << 24);
-      c += (key[offset + 8] & BYTE_MASK);
-      c += ((key[offset + 9] & BYTE_MASK) <<  8);
-      c += ((key[offset + 10] & BYTE_MASK) << 16);
-      c += ((key[offset + 11] & BYTE_MASK) << 24);
+      a += (hashKey.get(offset) & BYTE_MASK);
+      a += ((hashKey.get(offset + 1) & BYTE_MASK) <<  8);
+      a += ((hashKey.get(offset + 2) & BYTE_MASK) << 16);
+      a += ((hashKey.get(offset + 3) & BYTE_MASK) << 24);
+      b += (hashKey.get(offset + 4) & BYTE_MASK);
+      b += ((hashKey.get(offset + 5) & BYTE_MASK) <<  8);
+      b += ((hashKey.get(offset + 6) & BYTE_MASK) << 16);
+      b += ((hashKey.get(offset + 7) & BYTE_MASK) << 24);
+      c += (hashKey.get(offset + 8) & BYTE_MASK);
+      c += ((hashKey.get(offset + 9) & BYTE_MASK) <<  8);
+      c += ((hashKey.get(offset + 10) & BYTE_MASK) << 16);
+      c += ((hashKey.get(offset + 11) & BYTE_MASK) << 24);
 
       /*
        * mix -- mix 3 32-bit values reversibly.
@@ -164,30 +188,30 @@ public class JenkinsHash extends Hash {
     //-------------------------------- last block: affect all 32 bits of (c)
     switch (length) {                   // all the case statements fall through
     case 12:
-      c += ((key[offset + 11] & BYTE_MASK) << 24);
+      c += ((hashKey.get(offset + 11) & BYTE_MASK) << 24);
     case 11:
-      c += ((key[offset + 10] & BYTE_MASK) << 16);
+      c += ((hashKey.get(offset + 10) & BYTE_MASK) << 16);
     case 10:
-      c += ((key[offset + 9] & BYTE_MASK) <<  8);
+      c += ((hashKey.get(offset + 9) & BYTE_MASK) <<  8);
     case  9:
-      c += (key[offset + 8] & BYTE_MASK);
+      c += (hashKey.get(offset + 8) & BYTE_MASK);
     case  8:
-      b += ((key[offset + 7] & BYTE_MASK) << 24);
+      b += ((hashKey.get(offset + 7) & BYTE_MASK) << 24);
     case  7:
-      b += ((key[offset + 6] & BYTE_MASK) << 16);
+      b += ((hashKey.get(offset + 6) & BYTE_MASK) << 16);
     case  6:
-      b += ((key[offset + 5] & BYTE_MASK) <<  8);
+      b += ((hashKey.get(offset + 5) & BYTE_MASK) <<  8);
     case  5:
-      b += (key[offset + 4] & BYTE_MASK);
+      b += (hashKey.get(offset + 4) & BYTE_MASK);
     case  4:
-      a += ((key[offset + 3] & BYTE_MASK) << 24);
+      a += ((hashKey.get(offset + 3) & BYTE_MASK) << 24);
     case  3:
-      a += ((key[offset + 2] & BYTE_MASK) << 16);
+      a += ((hashKey.get(offset + 2) & BYTE_MASK) << 16);
     case  2:
-      a += ((key[offset + 1] & BYTE_MASK) <<  8);
+      a += ((hashKey.get(offset + 1) & BYTE_MASK) <<  8);
     case  1:
       //noinspection PointlessArithmeticExpression
-      a += (key[offset + 0] & BYTE_MASK);
+      a += (hashKey.get(offset + 0) & BYTE_MASK);
       break;
     case  0:
       return c;
@@ -238,28 +262,4 @@ public class JenkinsHash extends Hash {
     c ^= b; c -= rotateLeft(b, 24);
     return c;
   }
-
-  /**
-   * Compute the hash of the specified file
-   * @param args name of file to compute hash of.
-   * @throws IOException e
-   */
-  public static void main(String[] args) throws IOException {
-    if (args.length != 1) {
-      System.err.println("Usage: JenkinsHash filename");
-      System.exit(-1);
-    }
-    FileInputStream in = new FileInputStream(args[0]);
-    byte[] bytes = new byte[512];
-    int value = 0;
-    JenkinsHash hash = new JenkinsHash();
-    try {
-      for (int length = in.read(bytes); length > 0; length = in.read(bytes)) {
-        value = hash.hash(bytes, length, value);
-      }
-    } finally {
-      in.close();
-    }
-    System.out.println(Math.abs(value));
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/285e1041/hbase-common/src/main/java/org/apache/hadoop/hbase/util/MurmurHash.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/MurmurHash.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/MurmurHash.java
index 5c27386..86f0875 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/MurmurHash.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/MurmurHash.java
@@ -39,7 +39,7 @@ public class MurmurHash extends Hash {
   }
 
   @Override
-  public int hash(byte[] data, int offset, int length, int seed) {
+  public int hash(HashKey hashKey, int offset, int length, int seed) {
     int m = 0x5bd1e995;
     int r = 24;
 
@@ -49,14 +49,14 @@ public class MurmurHash extends Hash {
 
     for (int i = 0; i < len_4; i++) {
       int i_4 = (i << 2) + offset;
-      int k = data[i_4 + 3];
+      int k = hashKey.get(i_4 + 3);
       k = k << 8;
-      k = k | (data[i_4 + 2] & 0xff);
+      k = k | (hashKey.get(i_4 + 2) & 0xff);
       k = k << 8;
-      k = k | (data[i_4 + 1] & 0xff);
+      k = k | (hashKey.get(i_4 + 1) & 0xff);
       k = k << 8;
-      //noinspection PointlessArithmeticExpression
-      k = k | (data[i_4 + 0] & 0xff);
+      // noinspection PointlessArithmeticExpression
+      k = k | (hashKey.get(i_4 + 0) & 0xff);
       k *= m;
       k ^= k >>> r;
       k *= m;
@@ -71,13 +71,13 @@ public class MurmurHash extends Hash {
 
     if (left != 0) {
       if (left >= 3) {
-        h ^= data[i_m + 2] << 16;
+        h ^= hashKey.get(i_m + 2) << 16;
       }
       if (left >= 2) {
-        h ^= data[i_m + 1] << 8;
+        h ^= hashKey.get(i_m + 1) << 8;
       }
       if (left >= 1) {
-        h ^= data[i_m];
+        h ^= hashKey.get(i_m);
       }
 
       h *= m;

http://git-wip-us.apache.org/repos/asf/hbase/blob/285e1041/hbase-common/src/main/java/org/apache/hadoop/hbase/util/MurmurHash3.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/MurmurHash3.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/MurmurHash3.java
index 78d1331..22a23ae 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/MurmurHash3.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/MurmurHash3.java
@@ -40,7 +40,7 @@ public class MurmurHash3 extends Hash {
   /** Returns the MurmurHash3_x86_32 hash. */
   @edu.umd.cs.findbugs.annotations.SuppressWarnings("SF")
   @Override
-  public int hash(byte[] bytes, int offset, int length, int initval) {
+  public int hash(HashKey hashKey, int offset, int length, int initval) {
     final int c1 = 0xcc9e2d51;
     final int c2 = 0x1b873593;
 
@@ -49,8 +49,10 @@ public class MurmurHash3 extends Hash {
 
     for (int i = offset; i < roundedEnd; i += 4) {
       // little endian load order
-      int k1 = (bytes[i] & 0xff) | ((bytes[i + 1] & 0xff) << 8) | ((bytes[i + 2] & 0xff) << 16)
-          | (bytes[i + 3] << 24);
+      int k1 =
+          (hashKey.get(i) & 0xff) | ((hashKey.get(i + 1) & 0xff) << 8)
+              | ((hashKey.get(i + 2) & 0xff) << 16)
+              | (hashKey.get(i + 3) << 24);
       k1 *= c1;
       k1 = (k1 << 15) | (k1 >>> 17); // ROTL32(k1,15);
       k1 *= c2;
@@ -65,13 +67,13 @@ public class MurmurHash3 extends Hash {
 
     switch (length & 0x03) {
     case 3:
-      k1 = (bytes[roundedEnd + 2] & 0xff) << 16;
+      k1 = (hashKey.get(roundedEnd + 2) & 0xff) << 16;
       // FindBugs SF_SWITCH_FALLTHROUGH
     case 2:
-      k1 |= (bytes[roundedEnd + 1] & 0xff) << 8;
+      k1 |= (hashKey.get(roundedEnd + 1) & 0xff) << 8;
       // FindBugs SF_SWITCH_FALLTHROUGH
     case 1:
-      k1 |= (bytes[roundedEnd] & 0xff);
+      k1 |= (hashKey.get(roundedEnd) & 0xff);
       k1 *= c1;
       k1 = (k1 << 15) | (k1 >>> 17); // ROTL32(k1,15);
       k1 *= c2;

http://git-wip-us.apache.org/repos/asf/hbase/blob/285e1041/hbase-common/src/main/java/org/apache/hadoop/hbase/util/RowBloomHashKey.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/RowBloomHashKey.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/RowBloomHashKey.java
new file mode 100644
index 0000000..5253bf9
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/RowBloomHashKey.java
@@ -0,0 +1,35 @@
+/**
+ * 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
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.util;
+
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+@InterfaceAudience.Private
+public class RowBloomHashKey extends CellHashKey {
+
+  public RowBloomHashKey(Cell cell) {
+    super(cell);
+  }
+
+  @Override
+  public byte get(int offset) {
+    return CellUtil.getRowByte(t, offset);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/285e1041/hbase-common/src/main/java/org/apache/hadoop/hbase/util/RowColBloomHashKey.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/RowColBloomHashKey.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/RowColBloomHashKey.java
new file mode 100644
index 0000000..c35f307
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/RowColBloomHashKey.java
@@ -0,0 +1,80 @@
+/**
+ * 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
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.util;
+
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * An hash key for ROWCOL bloom. This assumes the cells to be serialized in the Keyvalue
+ * serialization format with Empty column family. Note that the byte representing the family length
+ * is considered to be 0
+ */
+@InterfaceAudience.Private
+public class RowColBloomHashKey extends CellHashKey {
+
+  private final int rowLength;
+  private final int qualLength;
+
+  public RowColBloomHashKey(Cell cell) {
+    super(cell);
+    rowLength = cell.getRowLength();
+    // We don't consider the family length for ROWCOL bloom. So subtract the famLen from the
+    // length calculation. Timestamp and type are of no relevance here
+    qualLength = cell.getQualifierLength();
+  }
+
+  @Override
+  public byte get(int offset) {
+    // Always assume that this cell has keyvalue serialized key structure.
+    // rk len + row key + 0 byte for family length + qual + ts + type
+    if (offset < Bytes.SIZEOF_SHORT) {
+      // assign locally
+      int rowlen = rowLength;
+      byte b = (byte) rowlen;
+      if (offset == 0) {
+        rowlen >>= 8;
+        b = (byte) rowlen;
+      }
+      return b;
+    }
+    int refLen = Bytes.SIZEOF_SHORT + rowLength;
+    if (offset < refLen) {
+      return CellUtil.getRowByte(t, offset - Bytes.SIZEOF_SHORT);
+    }
+    if (offset == refLen) {
+      // The fam length should return 0 assuming there is no column family.
+      // Because for ROWCOL blooms family is not considered
+      return 0;
+    }
+    refLen += qualLength + Bytes.SIZEOF_BYTE;
+    // skip the family len because actual cells may have family also
+    if (offset < refLen) {
+      return CellUtil.getQualifierByte(t,
+        offset - (Bytes.SIZEOF_SHORT + rowLength + Bytes.SIZEOF_BYTE));
+    }
+    // TODO : check if ts and type can be removed
+    refLen += KeyValue.TIMESTAMP_SIZE;
+    if (offset < refLen) {
+      return LATEST_TS[offset - (Bytes.SIZEOF_SHORT + rowLength + qualLength + Bytes.SIZEOF_BYTE)];
+    }
+    return MAX_TYPE;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/285e1041/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CompoundBloomFilter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CompoundBloomFilter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CompoundBloomFilter.java
index 2d773bb..79e8e0c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CompoundBloomFilter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CompoundBloomFilter.java
@@ -150,9 +150,11 @@ public class CompoundBloomFilter extends CompoundBloomFilterBase
     // Atleast we now avoid multiple copies until it comes here. If we want to make this to work
     // with BBs then the Hash.java APIs should also be changed to work with BBs.
     if (keyCell instanceof KeyValue) {
+      // TODO : directly use Cell here
       return checkContains(((KeyValue) keyCell).getBuffer(), ((KeyValue) keyCell).getKeyOffset(),
         ((KeyValue) keyCell).getKeyLength(), block);
     }
+    // TODO : Avoid this copy in read path also
     byte[] key = CellUtil.getCellKeySerializedAsKeyValueKey(keyCell);
     return checkContains(key, 0, key.length, block);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/285e1041/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CompoundBloomFilterWriter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CompoundBloomFilterWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CompoundBloomFilterWriter.java
index d95aafd..3193a17 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CompoundBloomFilterWriter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CompoundBloomFilterWriter.java
@@ -21,15 +21,16 @@ package org.apache.hadoop.hbase.io.hfile;
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
-import java.util.Arrays;
 import java.util.LinkedList;
 import java.util.Queue;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
-import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.regionserver.BloomType;
 import org.apache.hadoop.hbase.util.BloomFilterChunk;
 import org.apache.hadoop.hbase.util.BloomFilterUtil;
 import org.apache.hadoop.hbase.util.BloomFilterWriter;
@@ -78,6 +79,8 @@ public class CompoundBloomFilterWriter extends CompoundBloomFilterBase
   /** Whether to cache-on-write compound Bloom filter chunks */
   private boolean cacheOnWrite;
 
+  private BloomType bloomType;
+
   /**
    * @param chunkByteSizeHint
    *          each chunk's size in bytes. The real chunk size might be different
@@ -88,10 +91,12 @@ public class CompoundBloomFilterWriter extends CompoundBloomFilterBase
    *          hash function type to use
    * @param maxFold
    *          maximum degree of folding allowed
+   * @param bloomType
+   *          the bloom type
    */
   public CompoundBloomFilterWriter(int chunkByteSizeHint, float errorRate,
       int hashType, int maxFold, boolean cacheOnWrite,
-      CellComparator comparator) {
+      CellComparator comparator, BloomType bloomType) {
     chunkByteSize = BloomFilterUtil.computeFoldableByteSize(
         chunkByteSizeHint * 8L, maxFold);
 
@@ -100,6 +105,7 @@ public class CompoundBloomFilterWriter extends CompoundBloomFilterBase
     this.maxFold = maxFold;
     this.cacheOnWrite = cacheOnWrite;
     this.comparator = comparator;
+    this.bloomType = bloomType;
   }
 
   @Override
@@ -152,16 +158,9 @@ public class CompoundBloomFilterWriter extends CompoundBloomFilterBase
     chunk = null;
   }
 
-  /**
-   * Adds a Bloom filter key. This key must be greater than the previous key,
-   * as defined by the comparator this compound Bloom filter is configured
-   * with. For efficiency, key monotonicity is not checked here. See
-   * {@link StoreFileWriter#append(
-   * org.apache.hadoop.hbase.Cell)} for the details of deduplication.
-   */
   @Override
-  public void add(byte[] bloomKey, int keyOffset, int keyLength) {
-    if (bloomKey == null)
+  public void add(Cell cell) {
+    if (cell == null)
       throw new NullPointerException();
 
     enqueueReadyChunk(false);
@@ -171,32 +170,39 @@ public class CompoundBloomFilterWriter extends CompoundBloomFilterBase
         throw new IllegalStateException("First key in chunk already set: "
             + Bytes.toStringBinary(firstKeyInChunk));
       }
-      firstKeyInChunk = Arrays.copyOfRange(bloomKey, keyOffset, keyOffset
-          + keyLength);
-
-      if (prevChunk == null) {
-        // First chunk
-        chunk = BloomFilterUtil.createBySize(chunkByteSize, errorRate,
-            hashType, maxFold);
+      // This will be done only once per chunk
+      if (bloomType == BloomType.ROW) {
+        firstKeyInChunk = CellUtil.copyRow(cell);
       } else {
-        // Use the same parameters as the last chunk, but a new array and
-        // a zero key count.
-        chunk = prevChunk.createAnother();
-      }
-
-      if (chunk.getKeyCount() != 0) {
-        throw new IllegalStateException("keyCount=" + chunk.getKeyCount()
-            + " > 0");
+        firstKeyInChunk =
+            CellUtil.getCellKeySerializedAsKeyValueKey(CellUtil.createFirstOnRowCol(cell));
       }
-
-      chunk.allocBloom();
-      ++numChunks;
+      allocateNewChunk();
     }
 
-    chunk.add(bloomKey, keyOffset, keyLength);
+    chunk.add(cell);
     ++totalKeyCount;
   }
 
+  private void allocateNewChunk() {
+    if (prevChunk == null) {
+      // First chunk
+      chunk = BloomFilterUtil.createBySize(chunkByteSize, errorRate,
+          hashType, maxFold, bloomType);
+    } else {
+      // Use the same parameters as the last chunk, but a new array and
+      // a zero key count.
+      chunk = prevChunk.createAnother();
+    }
+
+    if (chunk.getKeyCount() != 0) {
+      throw new IllegalStateException("keyCount=" + chunk.getKeyCount()
+          + " > 0");
+    }
+
+    chunk.allocBloom();
+    ++numChunks;
+  }
   @Override
   public void writeInlineBlock(DataOutput out) throws IOException {
     // We don't remove the chunk from the queue here, because we might need it

http://git-wip-us.apache.org/repos/asf/hbase/blob/285e1041/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java
index 442b90d..47a8b25 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java
@@ -18,7 +18,9 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
-import com.google.common.base.Preconditions;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -29,20 +31,20 @@ import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.io.hfile.HFileContext;
 import org.apache.hadoop.hbase.regionserver.compactions.Compactor;
+import org.apache.hadoop.hbase.util.BloomContext;
 import org.apache.hadoop.hbase.util.BloomFilterFactory;
 import org.apache.hadoop.hbase.util.BloomFilterWriter;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.RowBloomContext;
+import org.apache.hadoop.hbase.util.RowColBloomContext;
 import org.apache.hadoop.io.WritableUtils;
 
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.util.Arrays;
+import com.google.common.base.Preconditions;
 
 /**
  * A StoreFile writer.  Use this to read/write HBase Store Files. It is package
@@ -55,12 +57,10 @@ public class StoreFileWriter implements Compactor.CellSink {
   private final BloomFilterWriter generalBloomFilterWriter;
   private final BloomFilterWriter deleteFamilyBloomFilterWriter;
   private final BloomType bloomType;
-  private byte[] lastBloomKey;
-  private int lastBloomKeyOffset, lastBloomKeyLen;
-  private Cell lastCell = null;
   private long earliestPutTs = HConstants.LATEST_TIMESTAMP;
   private Cell lastDeleteFamilyCell = null;
   private long deleteFamilyCnt = 0;
+  private BloomContext bloomContext = null;
 
   /**
    * timeRangeTrackerSet is used to figure if we were passed a filled-out TimeRangeTracker or not.
@@ -73,7 +73,6 @@ public class StoreFileWriter implements Compactor.CellSink {
    final TimeRangeTracker timeRangeTracker;
 
   protected HFile.Writer writer;
-  private KeyValue.KeyOnlyKeyValue lastBloomKeyOnlyKV = null;
 
   /**
    * Creates an HFile.Writer that also write helpful meta data.
@@ -134,9 +133,6 @@ public class StoreFileWriter implements Compactor.CellSink {
 
     if (generalBloomFilterWriter != null) {
       this.bloomType = bloomType;
-      if(this.bloomType ==  BloomType.ROWCOL) {
-        lastBloomKeyOnlyKV = new KeyValue.KeyOnlyKeyValue();
-      }
       if (LOG.isTraceEnabled()) {
         LOG.trace("Bloom filter type for " + path + ": " + this.bloomType + ", " +
             generalBloomFilterWriter.getClass().getSimpleName());
@@ -218,87 +214,30 @@ public class StoreFileWriter implements Compactor.CellSink {
   private void appendGeneralBloomfilter(final Cell cell) throws IOException {
     if (this.generalBloomFilterWriter != null) {
       // only add to the bloom filter on a new, unique key
-      boolean newKey = true;
-      if (this.lastCell != null) {
-        switch(bloomType) {
-          case ROW:
-            newKey = ! CellUtil.matchingRows(cell, lastCell);
-            break;
-          case ROWCOL:
-            newKey = ! CellUtil.matchingRowColumn(cell, lastCell);
-            break;
-          case NONE:
-            newKey = false;
-            break;
-          default:
-            throw new IOException("Invalid Bloom filter type: " + bloomType +
-                " (ROW or ROWCOL expected)");
-        }
-      }
-      if (newKey) {
-        /*
-         * http://2.bp.blogspot.com/_Cib_A77V54U/StZMrzaKufI/AAAAAAAAADo/ZhK7bGoJdMQ/s400/KeyValue.png
-         * Key = RowLen + Row + FamilyLen + Column [Family + Qualifier] + TimeStamp
-         *
-         * 2 Types of Filtering:
-         *  1. Row = Row
-         *  2. RowCol = Row + Qualifier
-         */
-        byte[] bloomKey = null;
-        // Used with ROW_COL bloom
-        KeyValue bloomKeyKV = null;
-        int bloomKeyOffset, bloomKeyLen;
-
+      if (this.bloomContext == null) {
+        // init bloom context
         switch (bloomType) {
-          case ROW:
-            bloomKey = cell.getRowArray();
-            bloomKeyOffset = cell.getRowOffset();
-            bloomKeyLen = cell.getRowLength();
-            break;
-          case ROWCOL:
-            // merge(row, qualifier)
-            // TODO: could save one buffer copy in case of compound Bloom
-            // filters when this involves creating a KeyValue
-            // TODO : Handle while writes also
-            bloomKeyKV = KeyValueUtil.createFirstOnRow(cell.getRowArray(), cell.getRowOffset(),
-                cell.getRowLength(),
-                HConstants.EMPTY_BYTE_ARRAY, 0, 0, cell.getQualifierArray(),
-                cell.getQualifierOffset(),
-                cell.getQualifierLength());
-            bloomKey = bloomKeyKV.getBuffer();
-            bloomKeyOffset = bloomKeyKV.getKeyOffset();
-            bloomKeyLen = bloomKeyKV.getKeyLength();
-            break;
-          default:
-            throw new IOException("Invalid Bloom filter type: " + bloomType +
-                " (ROW or ROWCOL expected)");
+        case ROW:
+          bloomContext = new RowBloomContext(generalBloomFilterWriter);
+          break;
+        case ROWCOL:
+          bloomContext = new RowColBloomContext(generalBloomFilterWriter);
+          break;
+        default:
+          throw new IOException(
+              "Invalid Bloom filter type: " + bloomType + " (ROW or ROWCOL expected)");
         }
-        generalBloomFilterWriter.add(bloomKey, bloomKeyOffset, bloomKeyLen);
-        if (lastBloomKey != null) {
-          int res = 0;
-          // hbase:meta does not have blooms. So we need not have special interpretation
-          // of the hbase:meta cells.  We can safely use Bytes.BYTES_RAWCOMPARATOR for ROW Bloom
-          if (bloomType == BloomType.ROW) {
-            res = Bytes.BYTES_RAWCOMPARATOR.compare(bloomKey, bloomKeyOffset, bloomKeyLen,
-                lastBloomKey, lastBloomKeyOffset, lastBloomKeyLen);
-          } else {
-            // TODO : Caching of kv components becomes important in these cases
-            res = CellComparator.COMPARATOR.compare(bloomKeyKV, lastBloomKeyOnlyKV);
-          }
-          if (res <= 0) {
-            throw new IOException("Non-increasing Bloom keys: "
-                + Bytes.toStringBinary(bloomKey, bloomKeyOffset, bloomKeyLen) + " after "
-                + Bytes.toStringBinary(lastBloomKey, lastBloomKeyOffset, lastBloomKeyLen));
-          }
-        }
-        lastBloomKey = bloomKey;
-        lastBloomKeyOffset = bloomKeyOffset;
-        lastBloomKeyLen = bloomKeyLen;
-        if (bloomType == BloomType.ROWCOL) {
-          lastBloomKeyOnlyKV.setKey(bloomKey, bloomKeyOffset, bloomKeyLen);
-        }
-        this.lastCell = cell;
       }
+
+      /*
+       * http://2.bp.blogspot.com/_Cib_A77V54U/StZMrzaKufI/AAAAAAAAADo/ZhK7bGoJdMQ/s400/KeyValue.png
+       * Key = RowLen + Row + FamilyLen + Column [Family + Qualifier] + TimeStamp
+       *
+       * 2 Types of Filtering:
+       *  1. Row = Row
+       *  2. RowCol = Row + Qualifier
+       */
+      bloomContext.writeBloom(cell);
     }
   }
 
@@ -317,9 +256,9 @@ public class StoreFileWriter implements Compactor.CellSink {
         // of the hbase:meta cells
         newKey = !CellUtil.matchingRows(cell, lastDeleteFamilyCell);
       }
+      // TODO : Use bloom context for delete family bloom filter also
       if (newKey) {
-        this.deleteFamilyBloomFilterWriter.add(cell.getRowArray(),
-            cell.getRowOffset(), cell.getRowLength());
+        this.deleteFamilyBloomFilterWriter.add(cell);
         this.lastDeleteFamilyCell = cell;
       }
     }
@@ -365,11 +304,7 @@ public class StoreFileWriter implements Compactor.CellSink {
       writer.addGeneralBloomFilter(generalBloomFilterWriter);
       writer.appendFileInfo(StoreFile.BLOOM_FILTER_TYPE_KEY,
           Bytes.toBytes(bloomType.toString()));
-      if (lastBloomKey != null) {
-        writer.appendFileInfo(StoreFile.LAST_BLOOM_KEY, Arrays.copyOfRange(
-            lastBloomKey, lastBloomKeyOffset, lastBloomKeyOffset
-                + lastBloomKeyLen));
-      }
+      bloomContext.addLastBloomKey(writer);
     }
     return hasGeneralBloom;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/285e1041/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomContext.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomContext.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomContext.java
new file mode 100644
index 0000000..aab30e0
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomContext.java
@@ -0,0 +1,71 @@
+/**
+ * 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
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.util;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.io.hfile.HFile;
+
+/**
+ * The bloom context that is used by the StorefileWriter to add the bloom details
+ * per cell
+ */
+@InterfaceAudience.Private
+public abstract class BloomContext {
+
+  // TODO : Avoid holding references to lastCell
+  protected Cell lastCell;
+
+  protected BloomFilterWriter generalBloomFilterWriter;
+
+  public BloomContext(BloomFilterWriter generalBloomFilterWriter) {
+    this.generalBloomFilterWriter = generalBloomFilterWriter;
+  }
+
+  public Cell getLastCell() {
+    return this.lastCell;
+  }
+
+  /**
+   * Bloom information from the cell is retrieved
+   * @param cell
+   * @throws IOException
+   */
+  public void writeBloom(Cell cell) throws IOException {
+    if (isNewKey(cell)) {
+      generalBloomFilterWriter.add(cell);
+      this.lastCell = cell;
+    }
+  }
+
+  /**
+   * Adds the last bloom key to the HFile Writer as part of StorefileWriter close.
+   * @param writer
+   * @throws IOException
+   */
+  public abstract void addLastBloomKey(HFile.Writer writer) throws IOException;
+
+  /**
+   * Returns true if the cell is a new key as per the bloom type
+   * @param cell the cell to be verified
+   * @return true if a new key else false
+   */
+  protected abstract boolean isNewKey(Cell cell);
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/285e1041/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterChunk.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterChunk.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterChunk.java
index 1e77984..de09938 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterChunk.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterChunk.java
@@ -24,7 +24,10 @@ import java.io.DataOutput;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.regionserver.BloomType;
 
 /**
  * The basic building block for the {@link org.apache.hadoop.hbase.io.hfile.CompoundBloomFilter}
@@ -46,6 +49,8 @@ public class BloomFilterChunk implements BloomFilterBase {
   protected int maxKeys;
   /** Bloom bits */
   protected ByteBuffer bloom;
+  /** The type of bloom */
+  protected BloomType bloomType;
 
   /**
    * Loads bloom filter meta data from file input.
@@ -80,9 +85,10 @@ public class BloomFilterChunk implements BloomFilterBase {
     return BloomFilterUtil.actualErrorRate(keyCount, byteSize * 8, hashCount);
   }
 
-  public BloomFilterChunk(int hashType) {
+  public BloomFilterChunk(int hashType, BloomType bloomType) {
     this.hashType = hashType;
     this.hash = Hash.getInstance(hashType);
+    this.bloomType = bloomType;
   }
 
   /**
@@ -100,9 +106,10 @@ public class BloomFilterChunk implements BloomFilterBase {
    *          than maxKeys.
    * @throws IllegalArgumentException
    */
+  // Used only in testcases
   public BloomFilterChunk(int maxKeys, double errorRate, int hashType,
       int foldFactor) throws IllegalArgumentException {
-    this(hashType);
+    this(hashType, BloomType.ROW);
 
     long bitSize = BloomFilterUtil.computeBitSize(maxKeys, errorRate);
     hashCount = BloomFilterUtil.optimalFunctionCount(maxKeys, bitSize);
@@ -121,7 +128,7 @@ public class BloomFilterChunk implements BloomFilterBase {
    * @return a Bloom filter with the same configuration as this
    */
   public BloomFilterChunk createAnother() {
-    BloomFilterChunk bbf = new BloomFilterChunk(hashType);
+    BloomFilterChunk bbf = new BloomFilterChunk(hashType, this.bloomType);
     bbf.byteSize = byteSize;
     bbf.hashCount = hashCount;
     bbf.maxKeys = maxKeys;
@@ -173,6 +180,38 @@ public class BloomFilterChunk implements BloomFilterBase {
     int hash1 = this.hash.hash(buf, offset, len, 0);
     int hash2 = this.hash.hash(buf, offset, len, hash1);
 
+    setHashLoc(hash1, hash2);
+  }
+
+  public void add(Cell cell) {
+    /*
+     * For faster hashing, use combinatorial generation
+     * http://www.eecs.harvard.edu/~kirsch/pubs/bbbf/esa06.pdf
+     */
+    int hash1;
+    int hash2;
+    HashKey<Cell> hashKey;
+    if (this.bloomType == BloomType.ROW) {
+      // TODO : Move this length to the HashKey when we do the read path to work with
+      // extractor so that the byte[] version of hash() function is removed
+      int length = cell.getRowLength();
+      hashKey = new RowBloomHashKey(cell);
+      hash1 = this.hash.hash(hashKey, 0, length, 0);
+      hash2 = this.hash.hash(hashKey, 0, length, hash1);
+    } else {
+      int famLen = cell.getFamilyLength();
+      // TODO : Move this length to the HashKey when we do the read path to work with
+      // extractor so that the byte[] version of hash() function is removed
+      int length = KeyValueUtil.keyLength(cell) - famLen;
+      hashKey = new RowColBloomHashKey(cell);
+      hash1 = this.hash.hash(hashKey, 0, length, 0);
+      hash2 = this.hash.hash(hashKey, 0, length, hash1);
+    }
+
+    setHashLoc(hash1, hash2);
+  }
+
+  private void setHashLoc(int hash1, int hash2) {
     for (int i = 0; i < this.hashCount; i++) {
       long hashLoc = Math.abs((hash1 + i * hash2) % (this.byteSize * 8));
       set(hashLoc);

http://git-wip-us.apache.org/repos/asf/hbase/blob/285e1041/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterFactory.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterFactory.java
index 99951f0..22d6fe1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterFactory.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterFactory.java
@@ -196,7 +196,7 @@ public final class BloomFilterFactory {
     // In case of compound Bloom filters we ignore the maxKeys hint.
     CompoundBloomFilterWriter bloomWriter = new CompoundBloomFilterWriter(getBloomBlockSize(conf),
         err, Hash.getHashType(conf), maxFold, cacheConf.shouldCacheBloomsOnWrite(),
-        bloomType == BloomType.ROWCOL ? CellComparator.COMPARATOR : null);
+        bloomType == BloomType.ROWCOL ? CellComparator.COMPARATOR : null, bloomType);
     writer.addInlineBlockWriter(bloomWriter);
     return bloomWriter;
   }
@@ -227,7 +227,7 @@ public final class BloomFilterFactory {
     // In case of compound Bloom filters we ignore the maxKeys hint.
     CompoundBloomFilterWriter bloomWriter = new CompoundBloomFilterWriter(getBloomBlockSize(conf),
         err, Hash.getHashType(conf), maxFold, cacheConf.shouldCacheBloomsOnWrite(),
-        null);
+        null, BloomType.ROW);
     writer.addInlineBlockWriter(bloomWriter);
     return bloomWriter;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/285e1041/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterUtil.java
index 08ed753..4b4e87d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterUtil.java
@@ -22,6 +22,7 @@ import java.util.Random;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.nio.ByteBuff;
+import org.apache.hadoop.hbase.regionserver.BloomType;
 
 /**
  * Utility methods related to BloomFilters
@@ -173,11 +174,12 @@ public final class BloomFilterUtil {
    * @param errorRate target false positive rate of the Bloom filter
    * @param hashType Bloom filter hash function type
    * @param foldFactor
+   * @param bloomType
    * @return the new Bloom filter of the desired size
    */
   public static BloomFilterChunk createBySize(int byteSizeHint,
-      double errorRate, int hashType, int foldFactor) {
-    BloomFilterChunk bbf = new BloomFilterChunk(hashType);
+      double errorRate, int hashType, int foldFactor, BloomType bloomType) {
+    BloomFilterChunk bbf = new BloomFilterChunk(hashType, bloomType);
 
     bbf.byteSize = computeFoldableByteSize(byteSizeHint * 8L, foldFactor);
     long bitSize = bbf.byteSize * 8;
@@ -195,11 +197,12 @@ public final class BloomFilterUtil {
   public static boolean contains(byte[] buf, int offset, int length,
       ByteBuff bloomBuf, int bloomOffset, int bloomSize, Hash hash,
       int hashCount) {
-
-    int hash1 = hash.hash(buf, offset, length, 0);
-    int hash2 = hash.hash(buf, offset, length, hash1);
+    // TODO : this will get removed once read path also work with Cell for blooms.
+    ByteArrayHashKey hashKey = new ByteArrayHashKey(buf);
+    int hash1 = hash.hash(hashKey, offset, length, 0);
+    int hash2 = hash.hash(hashKey, offset, length, hash1);
     int bloomBitSize = bloomSize << 3;
-    
+
     if (randomGeneratorForTest == null) {
       // Production mode.
       int compositeHash = hash1;

http://git-wip-us.apache.org/repos/asf/hbase/blob/285e1041/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterWriter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterWriter.java
index 6869d69..32a9ff4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterWriter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterWriter.java
@@ -19,6 +19,7 @@
 
 package org.apache.hadoop.hbase.util;
 
+import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.io.Writable;
 
@@ -48,11 +49,7 @@ public interface BloomFilterWriter extends BloomFilterBase {
 
   /**
    * Add the specified binary to the bloom filter.
-   *
-   * @param buf data to be added to the bloom
-   * @param offset offset into the data to be added
-   * @param len length of the data to be added
+   * @param cell the cell data to be added to the bloom
    */
-  void add(byte[] buf, int offset, int len);
-
+  void add(Cell cell);
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/285e1041/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RowBloomContext.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RowBloomContext.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RowBloomContext.java
new file mode 100644
index 0000000..c61e840
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RowBloomContext.java
@@ -0,0 +1,52 @@
+/**
+ * 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
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.util;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.io.hfile.HFile.Writer;
+import org.apache.hadoop.hbase.regionserver.StoreFile;
+
+/**
+ * Handles ROW bloom related context. It works with both ByteBufferedCell and byte[] backed cells
+ */
+@InterfaceAudience.Private
+public class RowBloomContext extends BloomContext {
+
+  public RowBloomContext(BloomFilterWriter generalBloomFilterWriter) {
+    super(generalBloomFilterWriter);
+  }
+
+  public void addLastBloomKey(Writer writer) throws IOException {
+    if (lastCell != null) {
+      byte[] key = CellUtil.copyRow(this.lastCell);
+      writer.appendFileInfo(StoreFile.LAST_BLOOM_KEY, key);
+    }
+  }
+
+  @Override
+  protected boolean isNewKey(Cell cell) {
+    if (this.lastCell != null) {
+      return !CellUtil.matchingRows(cell, this.lastCell);
+    }
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/285e1041/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RowColBloomContext.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RowColBloomContext.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RowColBloomContext.java
new file mode 100644
index 0000000..c1b47af
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RowColBloomContext.java
@@ -0,0 +1,56 @@
+/**
+ * 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
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.util;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.io.hfile.HFile.Writer;
+import org.apache.hadoop.hbase.regionserver.StoreFile;
+
+/**
+ * Handles ROWCOL bloom related context. It can work with both BytebufferdCells
+ * and byte[] backed cells
+ */
+@InterfaceAudience.Private
+public class RowColBloomContext extends BloomContext {
+
+  public RowColBloomContext(BloomFilterWriter generalBloomFilterWriter) {
+    super(generalBloomFilterWriter);
+  }
+
+  @Override
+  public void addLastBloomKey(Writer writer) throws IOException {
+    if (this.lastCell != null) {
+      Cell firstOnRow = CellUtil.createFirstOnRowCol(this.lastCell);
+      // This copy happens only once when the writer is closed
+      byte[] key = CellUtil.getCellKeySerializedAsKeyValueKey(firstOnRow);
+      writer.appendFileInfo(StoreFile.LAST_BLOOM_KEY, key);
+    }
+  }
+
+  @Override
+  protected boolean isNewKey(Cell cell) {
+    if (this.lastCell != null) {
+      return !CellUtil.matchingRowColumn(cell, this.lastCell);
+    }
+    return true;
+  }
+}


[34/50] [abbrv] hbase git commit: HBASE-16368 test*WhenRegionMove in TestPartialResultsFromClientSide is flaky

Posted by sy...@apache.org.
HBASE-16368 test*WhenRegionMove in TestPartialResultsFromClientSide is flaky

Signed-off-by: zhangduo <zh...@apache.org>


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

Branch: refs/heads/hbase-12439
Commit: 091a17e3c94bf4a20f74c3d298d5a80d58c17300
Parents: ede9940
Author: Phil Yang <ud...@gmail.com>
Authored: Tue Aug 9 15:09:00 2016 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Wed Aug 10 18:13:12 2016 +0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/TestPartialResultsFromClientSide.java    | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/091a17e3/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java
index a0da9c2..5c4ca13 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java
@@ -100,7 +100,7 @@ public class TestPartialResultsFromClientSide {
   // getCellHeapSize().
   private static long CELL_HEAP_SIZE = -1;
 
-  private static long timeout = 2000;
+  private static long timeout = 10000;
 
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
@@ -1056,7 +1056,7 @@ public class TestPartialResultsFromClientSide {
     scan.setCaching(1);
     ResultScanner scanner = table.getScanner(scan);
     scanner.next();
-    Thread.sleep(timeout * 3);
+    Thread.sleep(timeout * 2);
     int count = 1;
     while (scanner.next() != null) {
       count++;


[33/50] [abbrv] hbase git commit: HBASE-16285 Drop RPC requests if it must be considered as timeout at client

Posted by sy...@apache.org.
HBASE-16285 Drop RPC requests if it must be considered as timeout at client

Signed-off-by: zhangduo <zh...@apache.org>


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

Branch: refs/heads/hbase-12439
Commit: ede9940a7bd254d496b2ef493d4f35540184e96a
Parents: 2d203e6
Author: Phil Yang <ud...@gmail.com>
Authored: Thu Aug 4 15:38:38 2016 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Wed Aug 10 16:56:27 2016 +0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/ipc/CallRunner.java |  7 +++-
 .../org/apache/hadoop/hbase/ipc/RpcServer.java  | 16 ++++---
 .../hadoop/hbase/ipc/RpcServerInterface.java    | 11 +++--
 .../org/apache/hadoop/hbase/client/TestHCM.java | 44 +++++++++++++++++++-
 4 files changed, 62 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/ede9940a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java
index e91699a..b2b3c66 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java
@@ -93,6 +93,11 @@ public class CallRunner {
         }
         return;
       }
+      call.startTime = System.currentTimeMillis();
+      if (call.startTime > call.deadline) {
+        RpcServer.LOG.info("Dropping timed out call: " + call);
+        return;
+      }
       this.status.setStatus("Setting up call");
       this.status.setConnection(call.connection.getHostAddress(), call.connection.getRemotePort());
       if (RpcServer.LOG.isTraceEnabled()) {
@@ -116,7 +121,7 @@ public class CallRunner {
         }
         // make the call
         resultPair = this.rpcServer.call(call.service, call.md, call.param, call.cellScanner,
-          call.timestamp, this.status, call.timeout);
+          call.timestamp, this.status, call.startTime, call.timeout);
       } catch (Throwable e) {
         RpcServer.LOG.debug(Thread.currentThread().getName() + ": " + call.toShortString(), e);
         errorThrowable = e;

http://git-wip-us.apache.org/repos/asf/hbase/blob/ede9940a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
index 73226aa..5b2aab1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
@@ -75,12 +75,9 @@ import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseIOException;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.Server;
-import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
-import org.apache.hadoop.hbase.client.Operation;
 import org.apache.hadoop.hbase.client.VersionInfoUtil;
 import org.apache.hadoop.hbase.codec.Codec;
 import org.apache.hadoop.hbase.conf.ConfigurationObserver;
@@ -99,7 +96,6 @@ import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse;
 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader;
 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader;
 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation;
-import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.security.AccessDeniedException;
 import org.apache.hadoop.hbase.security.AuthMethod;
 import org.apache.hadoop.hbase.security.HBasePolicyProvider;
@@ -312,6 +308,9 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
     protected long timestamp;      // the time received when response is null
                                    // the time served when response is not null
     protected int timeout;
+    protected long startTime;
+    protected long deadline;// the deadline to handle this call, if exceed we can drop it.
+
     /**
      * Chain of buffers to send as response.
      */
@@ -354,6 +353,7 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
       this.retryImmediatelySupported =
           connection == null? null: connection.retryImmediatelySupported;
       this.timeout = timeout;
+      this.deadline = this.timeout > 0 ? this.timestamp + this.timeout : Long.MAX_VALUE;
     }
 
     /**
@@ -1894,7 +1894,7 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
           ? new TraceInfo(header.getTraceInfo().getTraceId(), header.getTraceInfo().getParentId())
           : null;
       int timeout = 0;
-      if (header.hasTimeout()){
+      if (header.hasTimeout() && header.getTimeout() > 0){
         timeout = Math.max(minClientRequestTimeout, header.getTimeout());
       }
       Call call = new Call(id, this.service, md, header, param, cellScanner, this, responder,
@@ -2187,7 +2187,7 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
   public Pair<Message, CellScanner> call(BlockingService service, MethodDescriptor md,
       Message param, CellScanner cellScanner, long receiveTime, MonitoredRPCHandler status)
       throws IOException {
-    return call(service, md, param, cellScanner, receiveTime, status, 0);
+    return call(service, md, param, cellScanner, receiveTime, status, System.currentTimeMillis(),0);
   }
 
   /**
@@ -2195,10 +2195,9 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
    * the return response has protobuf response payload. On failure, the
    * exception name and the stack trace are returned in the protobuf response.
    */
-  @Override
   public Pair<Message, CellScanner> call(BlockingService service, MethodDescriptor md,
       Message param, CellScanner cellScanner, long receiveTime, MonitoredRPCHandler status,
-      int timeout)
+      long startTime, int timeout)
   throws IOException {
     try {
       status.setRPC(md.getName(), new Object[]{param}, receiveTime);
@@ -2206,7 +2205,6 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
       status.setRPCPacket(param);
       status.resume("Servicing call");
       //get an instance of the method arg type
-      long startTime = System.currentTimeMillis();
       PayloadCarryingRpcController controller = new PayloadCarryingRpcController(cellScanner);
       controller.setCallTimeout(timeout);
       Message result = service.callBlockingMethod(md, controller, param);

http://git-wip-us.apache.org/repos/asf/hbase/blob/ede9940a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServerInterface.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServerInterface.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServerInterface.java
index dd7e584..0388ea4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServerInterface.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServerInterface.java
@@ -48,14 +48,17 @@ public interface RpcServerInterface {
   void setSocketSendBufSize(int size);
   InetSocketAddress getListenerAddress();
 
+  /**
+   * @deprecated As of release 1.3, this will be removed in HBase 3.0
+   */
+  @Deprecated
   Pair<Message, CellScanner> call(BlockingService service, MethodDescriptor md,
     Message param, CellScanner cellScanner, long receiveTime, MonitoredRPCHandler status)
   throws IOException, ServiceException;
 
-  Pair<Message, CellScanner> call(BlockingService service, MethodDescriptor md,
-      Message param, CellScanner cellScanner, long receiveTime, MonitoredRPCHandler status,
-      int timeout)
-      throws IOException, ServiceException;
+  Pair<Message, CellScanner> call(BlockingService service, MethodDescriptor md, Message param,
+      CellScanner cellScanner, long receiveTime, MonitoredRPCHandler status, long startTime,
+      int timeout) throws IOException, ServiceException;
 
   void setErrorHandler(HBaseRPCErrorHandler handler);
   HBaseRPCErrorHandler getErrorHandler();

http://git-wip-us.apache.org/repos/asf/hbase/blob/ede9940a/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 4d47bde..bfd16a7 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
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hbase.client;
 
 import static org.junit.Assert.*;
+import com.google.common.collect.Lists;
 
 import java.io.IOException;
 import java.lang.reflect.Field;
@@ -78,7 +79,11 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestRule;
 
-import com.google.common.collect.Lists;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
 
 /**
  * This class is for testing HBaseConnectionManager features
@@ -105,6 +110,7 @@ public class TestHCM {
   private static final byte[] ROW = Bytes.toBytes("bbb");
   private static final byte[] ROW_X = Bytes.toBytes("xxx");
   private static Random _randy = new Random();
+  private static final int RPC_RETRY = 5;
 
 /**
 * This copro sleeps 20 second. The first call it fails. The second time, it works.
@@ -155,12 +161,31 @@ public class TestHCM {
     }
   }
 
+  public static class SleepLongerAtFirstCoprocessor extends BaseRegionObserver {
+    public static final int SLEEP_TIME = 2000;
+    static final AtomicLong ct = new AtomicLong(0);
+    @Override
+    public void preGetOp(final ObserverContext<RegionCoprocessorEnvironment> e,
+        final Get get, final List<Cell> results) throws IOException {
+      // After first sleep, all requests are timeout except the last retry. If we handle
+      // all the following requests, finally the last request is also timeout. If we drop all
+      // timeout requests, we can handle the last request immediately and it will not timeout.
+      if (ct.incrementAndGet() <= 1) {
+        Threads.sleep(SLEEP_TIME * RPC_RETRY * 2);
+      } else {
+        Threads.sleep(SLEEP_TIME);
+      }
+    }
+  }
+
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     TEST_UTIL.getConfiguration().setBoolean(HConstants.STATUS_PUBLISHED, true);
     // Up the handlers; this test needs more than usual.
     TEST_UTIL.getConfiguration().setInt(HConstants.REGION_SERVER_HIGH_PRIORITY_HANDLER_COUNT, 10);
-    TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 5);
+    TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, RPC_RETRY);
+    // simulate queue blocking in testDropTimeoutRequest
+    TEST_UTIL.getConfiguration().setInt(HConstants.REGION_SERVER_HANDLER_COUNT, 1);
     TEST_UTIL.startMiniCluster(2);
   }
 
@@ -442,6 +467,21 @@ public class TestHCM {
     }
   }
 
+  @Test
+  public void testDropTimeoutRequest() throws Exception {
+    // Simulate the situation that the server is slow and client retries for several times because
+    // of timeout. When a request can be handled after waiting in the queue, we will drop it if
+    // it has been considered as timeout at client. If we don't drop it, the server will waste time
+    // on handling timeout requests and finally all requests timeout and client throws exception.
+    HTableDescriptor hdt = TEST_UTIL.createTableDescriptor("HCM-testDropTimeputRequest");
+    hdt.addCoprocessor(SleepLongerAtFirstCoprocessor.class.getName());
+    Configuration c = new Configuration(TEST_UTIL.getConfiguration());
+    try (Table t = TEST_UTIL.createTable(hdt, new byte[][] { FAM_NAM }, c)) {
+      t.setRpcTimeout(SleepLongerAtFirstCoprocessor.SLEEP_TIME * 2);
+      t.get(new Get(FAM_NAM));
+    }
+  }
+
   /**
    * Test starting from 0 index when RpcRetryingCaller calculate the backoff time.
    */


[20/50] [abbrv] hbase git commit: Revert "HBASE-16308 Contain protobuf references Gather up the pb references into a few locations only rather than have pb references distributed all about the code base."

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/0206dc67/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 fa18bd8..29650ef 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
@@ -18,6 +18,10 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.ServiceException;
+
 import java.io.Closeable;
 import java.io.IOException;
 import java.io.InterruptedIOException;
@@ -28,7 +32,6 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
-import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
@@ -66,6 +69,7 @@ import org.apache.hadoop.hbase.ZooKeeperConnectionException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.client.security.SecurityCapability;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
 import org.apache.hadoop.hbase.ipc.MasterCoprocessorRpcChannel;
@@ -179,8 +183,6 @@ import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.zookeeper.KeeperException;
 
-import com.google.common.annotations.VisibleForTesting;
-
 /**
  * HBaseAdmin is no longer a client API. It is marked InterfaceAudience.Private indicating that
  * this is an HBase-internal class as defined in
@@ -209,6 +211,10 @@ public class HBaseAdmin implements Admin {
   private volatile Configuration conf;
   private final long pause;
   private final int numRetries;
+  // Some operations can take a long time such as disable of big table.
+  // numRetries is for 'normal' stuff... Multiply by this factor when
+  // want to wait a long time.
+  private final int retryLongerMultiplier;
   private final int syncWaitTimeout;
   private boolean aborted;
   private int operationTimeout;
@@ -233,6 +239,8 @@ public class HBaseAdmin implements Admin {
         HConstants.DEFAULT_HBASE_CLIENT_PAUSE);
     this.numRetries = this.conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
         HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
+    this.retryLongerMultiplier = this.conf.getInt(
+        "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,
@@ -254,7 +262,7 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public boolean isAborted() {
+  public boolean isAborted(){
     return this.aborted;
   }
 
@@ -266,16 +274,18 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public Future<Boolean> abortProcedureAsync(final long procId, final boolean mayInterruptIfRunning)
-  throws IOException {
+  public Future<Boolean> abortProcedureAsync(
+      final long procId,
+      final boolean mayInterruptIfRunning) throws IOException {
     Boolean abortProcResponse = executeCallable(
-      new MasterCallable<AbortProcedureResponse>(getConnection(), getRpcControllerFactory()) {
+      new MasterCallable<AbortProcedureResponse>(getConnection()) {
         @Override
-        protected AbortProcedureResponse call(PayloadCarryingRpcController rpcController)
-        throws Exception {
+        public AbortProcedureResponse call(int callTimeout) throws ServiceException {
+          PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+          controller.setCallTimeout(callTimeout);
           AbortProcedureRequest abortProcRequest =
               AbortProcedureRequest.newBuilder().setProcId(procId).build();
-          return master.abortProcedure(rpcController, abortProcRequest);
+          return master.abortProcedure(controller, abortProcRequest);
         }
       }).getIsProcedureAborted();
 
@@ -314,9 +324,9 @@ public class HBaseAdmin implements Admin {
 
   @Override
   public boolean tableExists(final TableName tableName) throws IOException {
-    return executeCallable(new RpcRetryingCallable<Boolean>() {
+    return executeCallable(new ConnectionCallable<Boolean>(getConnection()) {
       @Override
-      protected Boolean rpcCall(int callTimeout) throws Exception {
+      public Boolean call(int callTimeout) throws ServiceException, IOException {
         return MetaTableAccessor.tableExists(connection, tableName);
       }
     });
@@ -340,15 +350,14 @@ public class HBaseAdmin implements Admin {
   @Override
   public HTableDescriptor[] listTables(final Pattern pattern, final boolean includeSysTables)
       throws IOException {
-    return executeCallable(new MasterCallable<HTableDescriptor[]>(getConnection(),
-        getRpcControllerFactory()) {
+    return executeCallable(new MasterCallable<HTableDescriptor[]>(getConnection()) {
       @Override
-      protected HTableDescriptor[] call(PayloadCarryingRpcController rpcController)
-      throws Exception {
+      public HTableDescriptor[] call(int callTimeout) throws ServiceException {
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
         GetTableDescriptorsRequest req =
             RequestConverter.buildGetTableDescriptorsRequest(pattern, includeSysTables);
-        return ProtobufUtil.getHTableDescriptorArray(master.getTableDescriptors(rpcController,
-            req));
+        return ProtobufUtil.getHTableDescriptorArray(master.getTableDescriptors(controller, req));
       }
     });
   }
@@ -377,13 +386,14 @@ public class HBaseAdmin implements Admin {
   @Override
   public TableName[] listTableNames(final Pattern pattern, final boolean includeSysTables)
       throws IOException {
-    return executeCallable(new MasterCallable<TableName[]>(getConnection(),
-        getRpcControllerFactory()) {
+    return executeCallable(new MasterCallable<TableName[]>(getConnection()) {
       @Override
-      protected TableName[] call(PayloadCarryingRpcController rpcController) throws Exception {
+      public TableName[] call(int callTimeout) throws ServiceException {
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
         GetTableNamesRequest req =
             RequestConverter.buildGetTableNamesRequest(pattern, includeSysTables);
-        return ProtobufUtil.getTableNameArray(master.getTableNames(rpcController, req)
+        return ProtobufUtil.getTableNameArray(master.getTableNames(controller, req)
             .getTableNamesList());
       }
     });
@@ -404,25 +414,27 @@ public class HBaseAdmin implements Admin {
   static HTableDescriptor getTableDescriptor(final TableName tableName, Connection connection,
       RpcRetryingCallerFactory rpcCallerFactory, final RpcControllerFactory rpcControllerFactory,
       int operationTimeout, int rpcTimeout) throws IOException {
-    if (tableName == null) return null;
-    HTableDescriptor htd =
-        executeCallable(new MasterCallable<HTableDescriptor>(connection, rpcControllerFactory) {
-      @Override
-      protected HTableDescriptor call(PayloadCarryingRpcController rpcController)
-      throws Exception {
-        GetTableDescriptorsRequest req =
-            RequestConverter.buildGetTableDescriptorsRequest(tableName);
-        GetTableDescriptorsResponse htds = master.getTableDescriptors(rpcController, req);
-        if (!htds.getTableSchemaList().isEmpty()) {
-          return ProtobufUtil.convertToHTableDesc(htds.getTableSchemaList().get(0));
+      if (tableName == null) return null;
+      HTableDescriptor htd = executeCallable(new MasterCallable<HTableDescriptor>(connection) {
+        @Override
+        public HTableDescriptor call(int callTimeout) throws ServiceException {
+          PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+          controller.setCallTimeout(callTimeout);
+          GetTableDescriptorsResponse htds;
+          GetTableDescriptorsRequest req =
+                  RequestConverter.buildGetTableDescriptorsRequest(tableName);
+          htds = master.getTableDescriptors(controller, req);
+
+          if (!htds.getTableSchemaList().isEmpty()) {
+            return ProtobufUtil.convertToHTableDesc(htds.getTableSchemaList().get(0));
+          }
+          return null;
         }
-        return null;
+      }, rpcCallerFactory, operationTimeout, rpcTimeout);
+      if (htd != null) {
+        return htd;
       }
-    }, rpcCallerFactory, operationTimeout, rpcTimeout);
-    if (htd != null) {
-      return htd;
-    }
-    throw new TableNotFoundException(tableName.getNameAsString());
+      throw new TableNotFoundException(tableName.getNameAsString());
   }
 
   private long getPauseTime(int tries) {
@@ -490,14 +502,15 @@ public class HBaseAdmin implements Admin {
     }
 
     CreateTableResponse response = executeCallable(
-      new MasterCallable<CreateTableResponse>(getConnection(), getRpcControllerFactory()) {
+      new MasterCallable<CreateTableResponse>(getConnection()) {
         @Override
-        protected CreateTableResponse call(PayloadCarryingRpcController rpcController)
-        throws Exception {
-          rpcController.setPriority(desc.getTableName());
+        public CreateTableResponse call(int callTimeout) throws ServiceException {
+          PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+          controller.setCallTimeout(callTimeout);
+          controller.setPriority(desc.getTableName());
           CreateTableRequest request = RequestConverter.buildCreateTableRequest(
             desc, splitKeys, ng.getNonceGroup(), ng.newNonce());
-          return master.createTable(rpcController, request);
+          return master.createTable(controller, request);
         }
       });
     return new CreateTableFuture(this, desc, splitKeys, response);
@@ -541,14 +554,15 @@ public class HBaseAdmin implements Admin {
   @Override
   public Future<Void> deleteTableAsync(final TableName tableName) throws IOException {
     DeleteTableResponse response = executeCallable(
-      new MasterCallable<DeleteTableResponse>(getConnection(), getRpcControllerFactory()) {
+      new MasterCallable<DeleteTableResponse>(getConnection()) {
         @Override
-        protected DeleteTableResponse call(PayloadCarryingRpcController rpcController)
-        throws Exception {
-          rpcController.setPriority(tableName);
+        public DeleteTableResponse call(int callTimeout) throws ServiceException {
+          PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+          controller.setCallTimeout(callTimeout);
+          controller.setPriority(tableName);
           DeleteTableRequest req =
               RequestConverter.buildDeleteTableRequest(tableName, ng.getNonceGroup(),ng.newNonce());
-          return master.deleteTable(rpcController,req);
+          return master.deleteTable(controller,req);
         }
       });
     return new DeleteTableFuture(this, tableName, response);
@@ -622,16 +636,16 @@ public class HBaseAdmin implements Admin {
   public Future<Void> truncateTableAsync(final TableName tableName, final boolean preserveSplits)
       throws IOException {
     TruncateTableResponse response =
-        executeCallable(new MasterCallable<TruncateTableResponse>(getConnection(),
-            getRpcControllerFactory()) {
+        executeCallable(new MasterCallable<TruncateTableResponse>(getConnection()) {
           @Override
-          protected TruncateTableResponse call(PayloadCarryingRpcController rpcController)
-          throws Exception {
-            rpcController.setPriority(tableName);
+          public TruncateTableResponse call(int callTimeout) throws ServiceException {
+            PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+            controller.setCallTimeout(callTimeout);
+            controller.setPriority(tableName);
             LOG.info("Started truncating " + tableName);
             TruncateTableRequest req = RequestConverter.buildTruncateTableRequest(
               tableName, preserveSplits, ng.getNonceGroup(), ng.newNonce());
-            return master.truncateTable(rpcController, req);
+            return master.truncateTable(controller, req);
           }
         });
     return new TruncateTableFuture(this, tableName, preserveSplits, response);
@@ -687,15 +701,17 @@ public class HBaseAdmin implements Admin {
   public Future<Void> enableTableAsync(final TableName tableName) throws IOException {
     TableName.isLegalFullyQualifiedTableName(tableName.getName());
     EnableTableResponse response = executeCallable(
-      new MasterCallable<EnableTableResponse>(getConnection(), getRpcControllerFactory()) {
+      new MasterCallable<EnableTableResponse>(getConnection()) {
         @Override
-        protected EnableTableResponse call(PayloadCarryingRpcController rpcController)
-        throws Exception {
-          rpcController.setPriority(tableName);
+        public EnableTableResponse call(int callTimeout) throws ServiceException {
+          PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+          controller.setCallTimeout(callTimeout);
+          controller.setPriority(tableName);
+
           LOG.info("Started enable of " + tableName);
           EnableTableRequest req =
               RequestConverter.buildEnableTableRequest(tableName, ng.getNonceGroup(),ng.newNonce());
-          return master.enableTable(rpcController,req);
+          return master.enableTable(controller,req);
         }
       });
     return new EnableTableFuture(this, tableName, response);
@@ -751,16 +767,18 @@ public class HBaseAdmin implements Admin {
   public Future<Void> disableTableAsync(final TableName tableName) throws IOException {
     TableName.isLegalFullyQualifiedTableName(tableName.getName());
     DisableTableResponse response = executeCallable(
-      new MasterCallable<DisableTableResponse>(getConnection(), getRpcControllerFactory()) {
+      new MasterCallable<DisableTableResponse>(getConnection()) {
         @Override
-        protected DisableTableResponse call(PayloadCarryingRpcController rpcController)
-        throws Exception {
-          rpcController.setPriority(tableName);
+        public DisableTableResponse call(int callTimeout) throws ServiceException {
+          PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+          controller.setCallTimeout(callTimeout);
+          controller.setPriority(tableName);
+
           LOG.info("Started disable of " + tableName);
           DisableTableRequest req =
               RequestConverter.buildDisableTableRequest(
                 tableName, ng.getNonceGroup(), ng.newNonce());
-          return master.disableTable(rpcController, req);
+          return master.disableTable(controller, req);
         }
       });
     return new DisableTableFuture(this, tableName, response);
@@ -809,9 +827,9 @@ public class HBaseAdmin implements Admin {
   @Override
   public boolean isTableEnabled(final TableName tableName) throws IOException {
     checkTableExists(tableName);
-    return executeCallable(new RpcRetryingCallable<Boolean>() {
+    return executeCallable(new ConnectionCallable<Boolean>(getConnection()) {
       @Override
-      protected Boolean rpcCall(int callTimeout) throws Exception {
+      public Boolean call(int callTimeout) throws ServiceException, IOException {
         TableState tableState = MetaTableAccessor.getTableState(connection, tableName);
         if (tableState == null)
           throw new TableNotFoundException(tableName);
@@ -838,15 +856,16 @@ public class HBaseAdmin implements Admin {
 
   @Override
   public Pair<Integer, Integer> getAlterStatus(final TableName tableName) throws IOException {
-    return executeCallable(new MasterCallable<Pair<Integer, Integer>>(getConnection(),
-        getRpcControllerFactory()) {
+    return executeCallable(new MasterCallable<Pair<Integer, Integer>>(getConnection()) {
       @Override
-      protected Pair<Integer, Integer> call(PayloadCarryingRpcController rpcController)
-      throws Exception {
-        rpcController.setPriority(tableName);
+      public Pair<Integer, Integer> call(int callTimeout) throws ServiceException {
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
+        controller.setPriority(tableName);
+
         GetSchemaAlterStatusRequest req = RequestConverter
             .buildGetSchemaAlterStatusRequest(tableName);
-        GetSchemaAlterStatusResponse ret = master.getSchemaAlterStatus(rpcController, req);
+        GetSchemaAlterStatusResponse ret = master.getSchemaAlterStatus(controller, req);
         Pair<Integer, Integer> pair = new Pair<>(ret.getYetToUpdateRegions(),
             ret.getTotalRegions());
         return pair;
@@ -875,16 +894,17 @@ public class HBaseAdmin implements Admin {
   public Future<Void> addColumnFamily(final TableName tableName,
       final HColumnDescriptor columnFamily) throws IOException {
     AddColumnResponse response =
-        executeCallable(new MasterCallable<AddColumnResponse>(getConnection(),
-            getRpcControllerFactory()) {
+        executeCallable(new MasterCallable<AddColumnResponse>(getConnection()) {
           @Override
-          protected AddColumnResponse call(PayloadCarryingRpcController rpcController)
-          throws Exception {
-            rpcController.setPriority(tableName);
+          public AddColumnResponse call(int callTimeout) throws ServiceException {
+            PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+            controller.setCallTimeout(callTimeout);
+            controller.setPriority(tableName);
+
             AddColumnRequest req =
                 RequestConverter.buildAddColumnRequest(tableName, columnFamily, ng.getNonceGroup(),
                   ng.newNonce());
-            return master.addColumn(rpcController, req);
+            return master.addColumn(controller, req);
           }
         });
     return new AddColumnFamilyFuture(this, tableName, response);
@@ -919,16 +939,17 @@ public class HBaseAdmin implements Admin {
   public Future<Void> deleteColumnFamily(final TableName tableName, final byte[] columnFamily)
       throws IOException {
     DeleteColumnResponse response =
-        executeCallable(new MasterCallable<DeleteColumnResponse>(getConnection(),
-            getRpcControllerFactory()) {
+        executeCallable(new MasterCallable<DeleteColumnResponse>(getConnection()) {
           @Override
-          protected DeleteColumnResponse call(PayloadCarryingRpcController rpcController)
-          throws Exception {
-            rpcController.setPriority(tableName);
+          public DeleteColumnResponse call(int callTimeout) throws ServiceException {
+            PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+            controller.setCallTimeout(callTimeout);
+            controller.setPriority(tableName);
+
             DeleteColumnRequest req =
                 RequestConverter.buildDeleteColumnRequest(tableName, columnFamily,
                   ng.getNonceGroup(), ng.newNonce());
-            master.deleteColumn(rpcController, req);
+            master.deleteColumn(controller, req);
             return null;
           }
         });
@@ -964,16 +985,17 @@ public class HBaseAdmin implements Admin {
   public Future<Void> modifyColumnFamily(final TableName tableName,
       final HColumnDescriptor columnFamily) throws IOException {
     ModifyColumnResponse response =
-        executeCallable(new MasterCallable<ModifyColumnResponse>(getConnection(),
-            getRpcControllerFactory()) {
+        executeCallable(new MasterCallable<ModifyColumnResponse>(getConnection()) {
           @Override
-          protected ModifyColumnResponse call(PayloadCarryingRpcController rpcController)
-          throws Exception {
-            rpcController.setPriority(tableName);
+          public ModifyColumnResponse call(int callTimeout) throws ServiceException {
+            PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+            controller.setCallTimeout(callTimeout);
+            controller.setPriority(tableName);
+
             ModifyColumnRequest req =
                 RequestConverter.buildModifyColumnRequest(tableName, columnFamily,
                   ng.getNonceGroup(), ng.newNonce());
-            master.modifyColumn(rpcController, req);
+            master.modifyColumn(controller, req);
             return null;
           }
         });
@@ -1022,26 +1044,28 @@ public class HBaseAdmin implements Admin {
   @Override
   public boolean closeRegionWithEncodedRegionName(final String encodedRegionName,
       final String serverName) throws IOException {
-   if (null == serverName || ("").equals(serverName.trim())) {
-      throw new IllegalArgumentException("The servername cannot be null or empty.");
+    if (null == serverName || ("").equals(serverName.trim())) {
+      throw new IllegalArgumentException(
+          "The servername cannot be null or empty.");
     }
-    final ServerName sn = ServerName.valueOf(serverName);
-    final AdminService.BlockingInterface admin = connection.getAdmin(sn);
-    final PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-    return executeCallable(new RpcRetryingCallable<Boolean>() {
-      @Override
-      protected Boolean rpcCall(int callTimeout) throws Exception {
-        controller.setCallTimeout(callTimeout);
-        CloseRegionRequest request =
-            RequestConverter.buildCloseRegionRequest(sn, encodedRegionName);
-        CloseRegionResponse response = admin.closeRegion(controller, request);
-        boolean closed = response.getClosed();
-        if (false == closed) {
-          LOG.error("Not able to close the region " + encodedRegionName + ".");
-        }
-        return closed;
+    ServerName sn = ServerName.valueOf(serverName);
+    AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
+    // Close the region without updating zk state.
+    CloseRegionRequest request =
+      RequestConverter.buildCloseRegionRequest(sn, encodedRegionName);
+    try {
+      PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+
+      // TODO: this does not do retries, it should. Set priority and timeout in controller
+      CloseRegionResponse response = admin.closeRegion(controller, request);
+      boolean isRegionClosed = response.getClosed();
+      if (false == isRegionClosed) {
+        LOG.error("Not able to close the region " + encodedRegionName + ".");
       }
-    });
+      return isRegionClosed;
+    } catch (ServiceException se) {
+      throw ProtobufUtil.getRemoteException(se);
+    }
   }
 
   @Override
@@ -1080,20 +1104,20 @@ public class HBaseAdmin implements Admin {
     if (regionServerPair.getSecond() == null) {
       throw new NoServerForRegionException(Bytes.toStringBinary(regionName));
     }
-    final HRegionInfo hRegionInfo = regionServerPair.getFirst();
+    HRegionInfo hRegionInfo = regionServerPair.getFirst();
     ServerName serverName = regionServerPair.getSecond();
-    final PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-    final AdminService.BlockingInterface admin = this.connection.getAdmin(serverName);
-    executeCallable(new RpcRetryingCallable<Void>() {
-      @Override
-      protected Void rpcCall(int callTimeout) throws Exception {
-        controller.setCallTimeout(callTimeout);
-        FlushRegionRequest request =
-            RequestConverter.buildFlushRegionRequest(hRegionInfo.getRegionName());
-        admin.flushRegion(controller, request);
-        return null;
-      }
-    });
+
+    PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+
+    AdminService.BlockingInterface admin = this.connection.getAdmin(serverName);
+    FlushRegionRequest request =
+        RequestConverter.buildFlushRegionRequest(hRegionInfo.getRegionName());
+    try {
+      // TODO: this does not do retries, it should. Set priority and timeout in controller
+      admin.flushRegion(controller, request);
+    } catch (ServiceException se) {
+      throw ProtobufUtil.getRemoteException(se);
+    }
   }
 
   /**
@@ -1244,45 +1268,67 @@ public class HBaseAdmin implements Admin {
   private void compact(final ServerName sn, final HRegionInfo hri,
       final boolean major, final byte [] family)
   throws IOException {
-    final PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-    final AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
-    executeCallable(new RpcRetryingCallable<Void>() {
-      @Override
-      protected Void rpcCall(int callTimeout) throws Exception {
-        controller.setCallTimeout(callTimeout);
-        CompactRegionRequest request =
-            RequestConverter.buildCompactRegionRequest(hri.getRegionName(), major, family);
-        admin.compactRegion(controller, request);
-        return null;
-      }
-    });
+    PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+    AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
+    CompactRegionRequest request =
+      RequestConverter.buildCompactRegionRequest(hri.getRegionName(), major, family);
+    try {
+      // TODO: this does not do retries, it should. Set priority and timeout in controller
+      admin.compactRegion(controller, request);
+    } catch (ServiceException se) {
+      throw ProtobufUtil.getRemoteException(se);
+    }
   }
 
   @Override
   public void move(final byte [] encodedRegionName, final byte [] destServerName)
-  throws IOException {
-    executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
+      throws IOException {
+
+    executeCallable(new MasterCallable<Void>(getConnection()) {
       @Override
-      protected Void call(PayloadCarryingRpcController rpcController) throws Exception {
-        rpcController.setPriority(encodedRegionName);
-        MoveRegionRequest request =
-            RequestConverter.buildMoveRegionRequest(encodedRegionName, destServerName);
-        master.moveRegion(rpcController, request);
+      public Void call(int callTimeout) throws ServiceException {
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
+        // Hard to know the table name, at least check if meta
+        if (isMetaRegion(encodedRegionName)) {
+          controller.setPriority(TableName.META_TABLE_NAME);
+        }
+
+        try {
+          MoveRegionRequest request =
+              RequestConverter.buildMoveRegionRequest(encodedRegionName, destServerName);
+            master.moveRegion(controller, request);
+        } catch (DeserializationException de) {
+          LOG.error("Could not parse destination server name: " + de);
+          throw new ServiceException(new DoNotRetryIOException(de));
+        }
         return null;
       }
     });
   }
 
+  private boolean isMetaRegion(final byte[] regionName) {
+    return Bytes.equals(regionName, HRegionInfo.FIRST_META_REGIONINFO.getRegionName())
+        || Bytes.equals(regionName, HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes());
+  }
+
   @Override
-  public void assign(final byte [] regionName) throws MasterNotRunningException,
+  public void assign(final byte[] regionName) throws MasterNotRunningException,
       ZooKeeperConnectionException, IOException {
-    executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
+    final byte[] toBeAssigned = getRegionName(regionName);
+    executeCallable(new MasterCallable<Void>(getConnection()) {
       @Override
-      protected Void call(PayloadCarryingRpcController rpcController) throws Exception {
-        rpcController.setPriority(regionName);
+      public Void call(int callTimeout) throws ServiceException {
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
+        // Hard to know the table name, at least check if meta
+        if (isMetaRegion(regionName)) {
+          controller.setPriority(TableName.META_TABLE_NAME);
+        }
+
         AssignRegionRequest request =
-            RequestConverter.buildAssignRegionRequest(getRegionName(regionName));
-        master.assignRegion(rpcController, request);
+          RequestConverter.buildAssignRegionRequest(toBeAssigned);
+        master.assignRegion(controller,request);
         return null;
       }
     });
@@ -1292,13 +1338,18 @@ public class HBaseAdmin implements Admin {
   public void unassign(final byte [] regionName, final boolean force)
   throws MasterNotRunningException, ZooKeeperConnectionException, IOException {
     final byte[] toBeUnassigned = getRegionName(regionName);
-    executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
+    executeCallable(new MasterCallable<Void>(getConnection()) {
       @Override
-      protected Void call(PayloadCarryingRpcController rpcController) throws Exception {
-        rpcController.setPriority(regionName);
+      public Void call(int callTimeout) throws ServiceException {
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
+        // Hard to know the table name, at least check if meta
+        if (isMetaRegion(regionName)) {
+          controller.setPriority(TableName.META_TABLE_NAME);
+        }
         UnassignRegionRequest request =
-            RequestConverter.buildUnassignRegionRequest(toBeUnassigned, force);
-        master.unassignRegion(rpcController, request);
+          RequestConverter.buildUnassignRegionRequest(toBeUnassigned, force);
+        master.unassignRegion(controller, request);
         return null;
       }
     });
@@ -1307,11 +1358,16 @@ public class HBaseAdmin implements Admin {
   @Override
   public void offline(final byte [] regionName)
   throws IOException {
-    executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
+    executeCallable(new MasterCallable<Void>(getConnection()) {
       @Override
-      protected Void call(PayloadCarryingRpcController rpcController) throws Exception {
-        rpcController.setPriority(regionName);
-        master.offlineRegion(rpcController, RequestConverter.buildOfflineRegionRequest(regionName));
+      public Void call(int callTimeout) throws ServiceException {
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
+        // Hard to know the table name, at least check if meta
+        if (isMetaRegion(regionName)) {
+          controller.setPriority(TableName.META_TABLE_NAME);
+        }
+        master.offlineRegion(controller, RequestConverter.buildOfflineRegionRequest(regionName));
         return null;
       }
     });
@@ -1320,44 +1376,56 @@ public class HBaseAdmin implements Admin {
   @Override
   public boolean setBalancerRunning(final boolean on, final boolean synchronous)
   throws IOException {
-    return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
+    return executeCallable(new MasterCallable<Boolean>(getConnection()) {
       @Override
-      protected Boolean call(PayloadCarryingRpcController rpcController) throws Exception {
+      public Boolean call(int callTimeout) throws ServiceException {
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
+
         SetBalancerRunningRequest req =
             RequestConverter.buildSetBalancerRunningRequest(on, synchronous);
-        return master.setBalancerRunning(rpcController, req).getPrevBalanceValue();
+        return master.setBalancerRunning(controller, req).getPrevBalanceValue();
       }
     });
   }
 
   @Override
   public boolean balancer() throws IOException {
-    return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
+    return executeCallable(new MasterCallable<Boolean>(getConnection()) {
       @Override
-      protected Boolean call(PayloadCarryingRpcController rpcController) throws Exception {
-        return master.balance(rpcController,
-            RequestConverter.buildBalanceRequest(false)).getBalancerRan();
+      public Boolean call(int callTimeout) throws ServiceException {
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
+
+        return master.balance(controller,
+          RequestConverter.buildBalanceRequest(false)).getBalancerRan();
       }
     });
   }
 
   @Override
   public boolean balancer(final boolean force) throws IOException {
-    return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
+    return executeCallable(new MasterCallable<Boolean>(getConnection()) {
       @Override
-      protected Boolean call(PayloadCarryingRpcController rpcController) throws Exception {
-        return master.balance(rpcController,
-            RequestConverter.buildBalanceRequest(force)).getBalancerRan();
+      public Boolean call(int callTimeout) throws ServiceException {
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
+
+        return master.balance(controller,
+          RequestConverter.buildBalanceRequest(force)).getBalancerRan();
       }
     });
   }
 
   @Override
   public boolean isBalancerEnabled() throws IOException {
-    return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
+    return executeCallable(new MasterCallable<Boolean>(getConnection()) {
       @Override
-      protected Boolean call(PayloadCarryingRpcController rpcController) throws Exception {
-        return master.isBalancerEnabled(rpcController,
+      public Boolean call(int callTimeout) throws ServiceException {
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
+
+        return master.isBalancerEnabled(controller,
           RequestConverter.buildIsBalancerEnabledRequest()).getEnabled();
       }
     });
@@ -1365,10 +1433,13 @@ public class HBaseAdmin implements Admin {
 
   @Override
   public boolean normalize() throws IOException {
-    return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
+    return executeCallable(new MasterCallable<Boolean>(getConnection()) {
       @Override
-      protected Boolean call(PayloadCarryingRpcController rpcController) throws Exception {
-        return master.normalize(rpcController,
+      public Boolean call(int callTimeout) throws ServiceException {
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
+
+        return master.normalize(controller,
           RequestConverter.buildNormalizeRequest()).getNormalizerRan();
       }
     });
@@ -1376,10 +1447,13 @@ public class HBaseAdmin implements Admin {
 
   @Override
   public boolean isNormalizerEnabled() throws IOException {
-    return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
+    return executeCallable(new MasterCallable<Boolean>(getConnection()) {
       @Override
-      protected Boolean call(PayloadCarryingRpcController rpcController) throws Exception {
-        return master.isNormalizerEnabled(rpcController,
+      public Boolean call(int callTimeout) throws ServiceException {
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
+
+        return master.isNormalizerEnabled(controller,
           RequestConverter.buildIsNormalizerEnabledRequest()).getEnabled();
       }
     });
@@ -1387,22 +1461,28 @@ public class HBaseAdmin implements Admin {
 
   @Override
   public boolean setNormalizerRunning(final boolean on) throws IOException {
-    return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
+    return executeCallable(new MasterCallable<Boolean>(getConnection()) {
       @Override
-      protected Boolean call(PayloadCarryingRpcController rpcController) throws Exception {
+      public Boolean call(int callTimeout) throws ServiceException {
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
+
         SetNormalizerRunningRequest req =
           RequestConverter.buildSetNormalizerRunningRequest(on);
-        return master.setNormalizerRunning(rpcController, req).getPrevNormalizerValue();
+        return master.setNormalizerRunning(controller, req).getPrevNormalizerValue();
       }
     });
   }
 
   @Override
   public boolean enableCatalogJanitor(final boolean enable) throws IOException {
-    return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
+    return executeCallable(new MasterCallable<Boolean>(getConnection()) {
       @Override
-      protected Boolean call(PayloadCarryingRpcController rpcController) throws Exception {
-        return master.enableCatalogJanitor(rpcController,
+      public Boolean call(int callTimeout) throws ServiceException {
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
+
+        return master.enableCatalogJanitor(controller,
           RequestConverter.buildEnableCatalogJanitorRequest(enable)).getPrevValue();
       }
     });
@@ -1410,10 +1490,13 @@ public class HBaseAdmin implements Admin {
 
   @Override
   public int runCatalogScan() throws IOException {
-    return executeCallable(new MasterCallable<Integer>(getConnection(), getRpcControllerFactory()) {
+    return executeCallable(new MasterCallable<Integer>(getConnection()) {
       @Override
-      protected Integer call(PayloadCarryingRpcController rpcController) throws Exception {
-        return master.runCatalogScan(rpcController,
+      public Integer call(int callTimeout) throws ServiceException {
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
+
+        return master.runCatalogScan(controller,
           RequestConverter.buildCatalogScanRequest()).getScanResult();
       }
     });
@@ -1421,10 +1504,13 @@ public class HBaseAdmin implements Admin {
 
   @Override
   public boolean isCatalogJanitorEnabled() throws IOException {
-    return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
+    return executeCallable(new MasterCallable<Boolean>(getConnection()) {
       @Override
-      protected Boolean call(PayloadCarryingRpcController rpcController) throws Exception {
-        return master.isCatalogJanitorEnabled(rpcController,
+      public Boolean call(int callTimeout) throws ServiceException {
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
+
+        return master.isCatalogJanitorEnabled(controller,
           RequestConverter.buildIsCatalogJanitorEnabledRequest()).getValue();
       }
     });
@@ -1530,19 +1616,25 @@ public class HBaseAdmin implements Admin {
     }
 
     DispatchMergingRegionsResponse response =
-        executeCallable(new MasterCallable<DispatchMergingRegionsResponse>(getConnection(),
-            getRpcControllerFactory()) {
+      executeCallable(new MasterCallable<DispatchMergingRegionsResponse>(getConnection()) {
       @Override
-      protected DispatchMergingRegionsResponse call(PayloadCarryingRpcController rpcController)
-          throws Exception {
-        DispatchMergingRegionsRequest request = RequestConverter
-            .buildDispatchMergingRegionsRequest(
+      public DispatchMergingRegionsResponse call(int callTimeout) throws ServiceException {
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
+
+        try {
+          DispatchMergingRegionsRequest request = RequestConverter
+              .buildDispatchMergingRegionsRequest(
                 encodedNameOfRegionA,
                 encodedNameOfRegionB,
                 forcible,
                 ng.getNonceGroup(),
                 ng.newNonce());
-        return master.dispatchMergingRegions(rpcController, request);
+          return master.dispatchMergingRegions(controller, request);
+        } catch (DeserializationException de) {
+          LOG.error("Could not parse destination server name: " + de);
+          throw new ServiceException(new DoNotRetryIOException(de));
+        }
       }
     });
     return new DispatchMergingRegionsFuture(this, tableName, response);
@@ -1654,17 +1746,21 @@ public class HBaseAdmin implements Admin {
       throw new IllegalArgumentException("the specified table name '" + tableName +
         "' doesn't match with the HTD one: " + htd.getTableName());
     }
+
     ModifyTableResponse response = executeCallable(
-      new MasterCallable<ModifyTableResponse>(getConnection(), getRpcControllerFactory()) {
+      new MasterCallable<ModifyTableResponse>(getConnection()) {
         @Override
-        protected ModifyTableResponse call(PayloadCarryingRpcController rpcController)
-        throws Exception {
-          rpcController.setPriority(tableName);
+        public ModifyTableResponse call(int callTimeout) throws ServiceException {
+          PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+          controller.setCallTimeout(callTimeout);
+          controller.setPriority(tableName);
+
           ModifyTableRequest request = RequestConverter.buildModifyTableRequest(
             tableName, htd, ng.getNonceGroup(), ng.newNonce());
-          return master.modifyTable(rpcController, request);
+          return master.modifyTable(controller, request);
         }
       });
+
     return new ModifyTableFuture(this, tableName, response);
   }
 
@@ -1779,9 +1875,9 @@ public class HBaseAdmin implements Admin {
    */
   private TableName checkTableExists(final TableName tableName)
       throws IOException {
-    return executeCallable(new RpcRetryingCallable<TableName>() {
+    return executeCallable(new ConnectionCallable<TableName>(getConnection()) {
       @Override
-      protected TableName rpcCall(int callTimeout) throws Exception {
+      public TableName call(int callTimeout) throws ServiceException, IOException {
         if (!MetaTableAccessor.tableExists(connection, tableName)) {
           throw new TableNotFoundException(tableName);
         }
@@ -1792,11 +1888,13 @@ public class HBaseAdmin implements Admin {
 
   @Override
   public synchronized void shutdown() throws IOException {
-    executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
+    executeCallable(new MasterCallable<Void>(getConnection()) {
       @Override
-      protected Void call(PayloadCarryingRpcController rpcController) throws Exception {
-        rpcController.setPriority(HConstants.HIGH_QOS);
-        master.shutdown(rpcController, ShutdownRequest.newBuilder().build());
+      public Void call(int callTimeout) throws ServiceException {
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
+        controller.setPriority(HConstants.HIGH_QOS);
+        master.shutdown(controller, ShutdownRequest.newBuilder().build());
         return null;
       }
     });
@@ -1804,11 +1902,13 @@ public class HBaseAdmin implements Admin {
 
   @Override
   public synchronized void stopMaster() throws IOException {
-    executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
+    executeCallable(new MasterCallable<Void>(getConnection()) {
       @Override
-      protected Void call(PayloadCarryingRpcController rpcController) throws Exception {
-        rpcController.setPriority(HConstants.HIGH_QOS);
-        master.stopMaster(rpcController, StopMasterRequest.newBuilder().build());
+      public Void call(int callTimeout) throws ServiceException {
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
+        controller.setPriority(HConstants.HIGH_QOS);
+        master.stopMaster(controller, StopMasterRequest.newBuilder().build());
         return null;
       }
     });
@@ -1819,41 +1919,43 @@ public class HBaseAdmin implements Admin {
   throws IOException {
     String hostname = Addressing.parseHostname(hostnamePort);
     int port = Addressing.parsePort(hostnamePort);
-    final AdminService.BlockingInterface admin =
+    AdminService.BlockingInterface admin =
       this.connection.getAdmin(ServerName.valueOf(hostname, port, 0));
-    executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
-      @Override
-      protected Void call(PayloadCarryingRpcController rpcController) throws Exception {
-        rpcController.setPriority(HConstants.HIGH_QOS);
-        StopServerRequest request = RequestConverter.buildStopServerRequest(
-            "Called by admin client " + this.connection.toString());
-        admin.stopServer(rpcController, request);
-        return null;
-      }
-    });
+    StopServerRequest request = RequestConverter.buildStopServerRequest(
+      "Called by admin client " + this.connection.toString());
+    PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+
+    controller.setPriority(HConstants.HIGH_QOS);
+    try {
+      // TODO: this does not do retries, it should. Set priority and timeout in controller
+      admin.stopServer(controller, request);
+    } catch (ServiceException se) {
+      throw ProtobufUtil.getRemoteException(se);
+    }
   }
 
   @Override
   public boolean isMasterInMaintenanceMode() throws IOException {
-    return executeCallable(new MasterCallable<IsInMaintenanceModeResponse>(getConnection(),
-        this.rpcControllerFactory) {
+    return executeCallable(new MasterCallable<IsInMaintenanceModeResponse>(getConnection()) {
       @Override
-      protected IsInMaintenanceModeResponse call(PayloadCarryingRpcController rpcController)
-      throws Exception {
-        return master.isMasterInMaintenanceMode(rpcController,
-            IsInMaintenanceModeRequest.newBuilder().build());
+      public IsInMaintenanceModeResponse call(int callTimeout) throws ServiceException {
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
+        return master.isMasterInMaintenanceMode(
+          controller, IsInMaintenanceModeRequest.newBuilder().build());
       }
     }).getInMaintenanceMode();
   }
 
   @Override
   public ClusterStatus getClusterStatus() throws IOException {
-    return executeCallable(new MasterCallable<ClusterStatus>(getConnection(),
-        this.rpcControllerFactory) {
+    return executeCallable(new MasterCallable<ClusterStatus>(getConnection()) {
       @Override
-      protected ClusterStatus call(PayloadCarryingRpcController rpcController) throws Exception {
+      public ClusterStatus call(int callTimeout) throws ServiceException {
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
         GetClusterStatusRequest req = RequestConverter.buildGetClusterStatusRequest();
-        return ProtobufUtil.convert(master.getClusterStatus(rpcController, req).getClusterStatus());
+        return ProtobufUtil.convert(master.getClusterStatus(controller, req).getClusterStatus());
       }
     });
   }
@@ -1894,16 +1996,19 @@ public class HBaseAdmin implements Admin {
   public Future<Void> createNamespaceAsync(final NamespaceDescriptor descriptor)
       throws IOException {
     CreateNamespaceResponse response =
-        executeCallable(new MasterCallable<CreateNamespaceResponse>(getConnection(),
-            getRpcControllerFactory()) {
-      @Override
-      protected CreateNamespaceResponse call(PayloadCarryingRpcController rpcController)
-      throws Exception {
-        return master.createNamespace(rpcController,
-          CreateNamespaceRequest.newBuilder().setNamespaceDescriptor(ProtobufUtil.
-              toProtoNamespaceDescriptor(descriptor)).build());
-      }
-    });
+        executeCallable(new MasterCallable<CreateNamespaceResponse>(getConnection()) {
+          @Override
+          public CreateNamespaceResponse call(int callTimeout) throws Exception {
+            PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+            controller.setCallTimeout(callTimeout);
+            // TODO: set priority based on NS?
+            return master.createNamespace(controller,
+              CreateNamespaceRequest.newBuilder()
+              .setNamespaceDescriptor(ProtobufUtil
+                .toProtoNamespaceDescriptor(descriptor)).build()
+                );
+          }
+        });
     return new NamespaceFuture(this, descriptor.getName(), response.getProcId()) {
       @Override
       public String getOperationType() {
@@ -1922,16 +2027,16 @@ public class HBaseAdmin implements Admin {
   public Future<Void> modifyNamespaceAsync(final NamespaceDescriptor descriptor)
       throws IOException {
     ModifyNamespaceResponse response =
-        executeCallable(new MasterCallable<ModifyNamespaceResponse>(getConnection(),
-            getRpcControllerFactory()) {
-      @Override
-      protected ModifyNamespaceResponse call(PayloadCarryingRpcController rpcController)
-      throws Exception {
-        // TODO: set priority based on NS?
-        return master.modifyNamespace(rpcController, ModifyNamespaceRequest.newBuilder().
-          setNamespaceDescriptor(ProtobufUtil.toProtoNamespaceDescriptor(descriptor)).build());
-       }
-    });
+        executeCallable(new MasterCallable<ModifyNamespaceResponse>(getConnection()) {
+          @Override
+          public ModifyNamespaceResponse call(int callTimeout) throws Exception {
+            PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+            controller.setCallTimeout(callTimeout);
+            // TODO: set priority based on NS?
+            return master.modifyNamespace(controller, ModifyNamespaceRequest.newBuilder().
+              setNamespaceDescriptor(ProtobufUtil.toProtoNamespaceDescriptor(descriptor)).build());
+          }
+        });
     return new NamespaceFuture(this, descriptor.getName(), response.getProcId()) {
       @Override
       public String getOperationType() {
@@ -1950,16 +2055,16 @@ public class HBaseAdmin implements Admin {
   public Future<Void> deleteNamespaceAsync(final String name)
       throws IOException {
     DeleteNamespaceResponse response =
-        executeCallable(new MasterCallable<DeleteNamespaceResponse>(getConnection(),
-            getRpcControllerFactory()) {
-      @Override
-      protected DeleteNamespaceResponse call(PayloadCarryingRpcController rpcController)
-      throws Exception {
-        // TODO: set priority based on NS?
-        return master.deleteNamespace(rpcController, DeleteNamespaceRequest.newBuilder().
-          setNamespaceName(name).build());
-        }
-      });
+        executeCallable(new MasterCallable<DeleteNamespaceResponse>(getConnection()) {
+          @Override
+          public DeleteNamespaceResponse call(int callTimeout) throws Exception {
+            PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+            controller.setCallTimeout(callTimeout);
+            // TODO: set priority based on NS?
+            return master.deleteNamespace(controller, DeleteNamespaceRequest.newBuilder().
+              setNamespaceName(name).build());
+          }
+        });
     return new NamespaceFuture(this, name, response.getProcId()) {
       @Override
       public String getOperationType() {
@@ -1970,94 +2075,100 @@ public class HBaseAdmin implements Admin {
 
   @Override
   public NamespaceDescriptor getNamespaceDescriptor(final String name) throws IOException {
-    return executeCallable(new MasterCallable<NamespaceDescriptor>(getConnection(),
-        getRpcControllerFactory()) {
-      @Override
-      protected NamespaceDescriptor call(PayloadCarryingRpcController rpcController)
-          throws Exception {
-        return ProtobufUtil.toNamespaceDescriptor(
-            master.getNamespaceDescriptor(rpcController, GetNamespaceDescriptorRequest.newBuilder().
+    return
+        executeCallable(new MasterCallable<NamespaceDescriptor>(getConnection()) {
+          @Override
+          public NamespaceDescriptor call(int callTimeout) throws Exception {
+            PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+            controller.setCallTimeout(callTimeout);
+            return ProtobufUtil.toNamespaceDescriptor(
+              master.getNamespaceDescriptor(controller, GetNamespaceDescriptorRequest.newBuilder().
                 setNamespaceName(name).build()).getNamespaceDescriptor());
-      }
-    });
+          }
+        });
   }
 
   @Override
   public NamespaceDescriptor[] listNamespaceDescriptors() throws IOException {
-    return executeCallable(new MasterCallable<NamespaceDescriptor[]>(getConnection(),
-        getRpcControllerFactory()) {
-      @Override
-      protected NamespaceDescriptor[] call(PayloadCarryingRpcController rpcController)
-          throws Exception {
-        List<HBaseProtos.NamespaceDescriptor> list =
-            master.listNamespaceDescriptors(rpcController,
-              ListNamespaceDescriptorsRequest.newBuilder().build()).getNamespaceDescriptorList();
-        NamespaceDescriptor[] res = new NamespaceDescriptor[list.size()];
-        for(int i = 0; i < list.size(); i++) {
-          res[i] = ProtobufUtil.toNamespaceDescriptor(list.get(i));
-        }
-        return res;
-      }
-    });
+    return
+        executeCallable(new MasterCallable<NamespaceDescriptor[]>(getConnection()) {
+          @Override
+          public NamespaceDescriptor[] call(int callTimeout) throws Exception {
+            PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+            controller.setCallTimeout(callTimeout);
+            List<HBaseProtos.NamespaceDescriptor> list =
+                master.listNamespaceDescriptors(controller,
+                  ListNamespaceDescriptorsRequest.newBuilder().build())
+                .getNamespaceDescriptorList();
+            NamespaceDescriptor[] res = new NamespaceDescriptor[list.size()];
+            for(int i = 0; i < list.size(); i++) {
+              res[i] = ProtobufUtil.toNamespaceDescriptor(list.get(i));
+            }
+            return res;
+          }
+        });
   }
 
   @Override
   public ProcedureInfo[] listProcedures() throws IOException {
-    return executeCallable(new MasterCallable<ProcedureInfo[]>(getConnection(),
-        getRpcControllerFactory()) {
-      @Override
-      protected ProcedureInfo[] call(PayloadCarryingRpcController rpcController)
-          throws Exception {
-        List<ProcedureProtos.Procedure> procList = master.listProcedures(
-            rpcController, ListProceduresRequest.newBuilder().build()).getProcedureList();
-        ProcedureInfo[] procInfoList = new ProcedureInfo[procList.size()];
-        for (int i = 0; i < procList.size(); i++) {
-          procInfoList[i] = ProcedureUtil.convert(procList.get(i));
-        }
-        return procInfoList;
-      }
-    });
+    return
+        executeCallable(new MasterCallable<ProcedureInfo[]>(getConnection()) {
+          @Override
+          public ProcedureInfo[] call(int callTimeout) throws Exception {
+            PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+            controller.setCallTimeout(callTimeout);
+            List<ProcedureProtos.Procedure> procList = master.listProcedures(
+              controller, ListProceduresRequest.newBuilder().build()).getProcedureList();
+            ProcedureInfo[] procInfoList = new ProcedureInfo[procList.size()];
+            for (int i = 0; i < procList.size(); i++) {
+              procInfoList[i] = ProcedureUtil.convert(procList.get(i));
+            }
+            return procInfoList;
+          }
+        });
   }
 
   @Override
   public HTableDescriptor[] listTableDescriptorsByNamespace(final String name) throws IOException {
-    return executeCallable(new MasterCallable<HTableDescriptor[]>(getConnection(),
-        getRpcControllerFactory()) {
-      @Override
-      protected HTableDescriptor[] call(PayloadCarryingRpcController rpcController)
-          throws Exception {
-        List<TableSchema> list =
-            master.listTableDescriptorsByNamespace(rpcController,
-                ListTableDescriptorsByNamespaceRequest.newBuilder().setNamespaceName(name)
-                .build()).getTableSchemaList();
-        HTableDescriptor[] res = new HTableDescriptor[list.size()];
-        for(int i=0; i < list.size(); i++) {
-
-          res[i] = ProtobufUtil.convertToHTableDesc(list.get(i));
-        }
-        return res;
-      }
-    });
+    return
+        executeCallable(new MasterCallable<HTableDescriptor[]>(getConnection()) {
+          @Override
+          public HTableDescriptor[] call(int callTimeout) throws Exception {
+            PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+            controller.setCallTimeout(callTimeout);
+            List<TableSchema> list =
+                master.listTableDescriptorsByNamespace(controller,
+                  ListTableDescriptorsByNamespaceRequest.newBuilder().setNamespaceName(name)
+                  .build()).getTableSchemaList();
+            HTableDescriptor[] res = new HTableDescriptor[list.size()];
+            for(int i=0; i < list.size(); i++) {
+
+              res[i] = ProtobufUtil.convertToHTableDesc(list.get(i));
+            }
+            return res;
+          }
+        });
   }
 
   @Override
   public TableName[] listTableNamesByNamespace(final String name) throws IOException {
-    return executeCallable(new MasterCallable<TableName[]>(getConnection(),
-        getRpcControllerFactory()) {
-      @Override
-      protected TableName[] call(PayloadCarryingRpcController rpcController)
-          throws Exception {
-        List<HBaseProtos.TableName> tableNames =
-            master.listTableNamesByNamespace(rpcController, ListTableNamesByNamespaceRequest.
+    return
+        executeCallable(new MasterCallable<TableName[]>(getConnection()) {
+          @Override
+          public TableName[] call(int callTimeout) throws Exception {
+            PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+            controller.setCallTimeout(callTimeout);
+            List<HBaseProtos.TableName> tableNames =
+              master.listTableNamesByNamespace(controller, ListTableNamesByNamespaceRequest.
                 newBuilder().setNamespaceName(name).build())
-            .getTableNameList();
-        TableName[] result = new TableName[tableNames.size()];
-        for (int i = 0; i < tableNames.size(); i++) {
-          result[i] = ProtobufUtil.toTableName(tableNames.get(i));
-        }
-        return result;
-      }
-    });
+                .getTableNameList();
+            TableName[] result = new TableName[tableNames.size()];
+            for (int i = 0; i < tableNames.size(); i++) {
+              result[i] = ProtobufUtil.toTableName(tableNames.get(i));
+            }
+            return result;
+          }
+        });
   }
 
   /**
@@ -2065,26 +2176,10 @@ public class HBaseAdmin implements Admin {
    * @param conf system configuration
    * @throws MasterNotRunningException if the master is not running
    * @throws ZooKeeperConnectionException if unable to connect to zookeeper
-   * @deprecated since hbase-2.0.0 because throws a ServiceException. We don't want to have
-   * protobuf as part of our public API. Use {@link #available(Configuration)}
    */
   // Used by tests and by the Merge tool. Merge tool uses it to figure if HBase is up or not.
-  // MOB uses it too.
-  // NOTE: hbase-2.0.0 removes ServiceException from the throw.
-  @Deprecated
   public static void checkHBaseAvailable(Configuration conf)
-  throws MasterNotRunningException, ZooKeeperConnectionException, IOException,
-  com.google.protobuf.ServiceException {
-    available(conf);
-  }
-
-  /**
-   * Is HBase available? Throw an exception if not.
-   * @param conf system configuration
-   * @throws ZooKeeperConnectionException if unable to connect to zookeeper]
-   */
-  public static void available(final Configuration conf)
-  throws ZooKeeperConnectionException, InterruptedIOException {
+  throws MasterNotRunningException, ZooKeeperConnectionException, ServiceException, IOException {
     Configuration copyOfConf = HBaseConfiguration.create(conf);
     // We set it to make it fail as soon as possible if HBase is not available
     copyOfConf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1);
@@ -2096,6 +2191,7 @@ public class HBaseAdmin implements Admin {
              (ClusterConnection) ConnectionFactory.createConnection(copyOfConf);
          ZooKeeperKeepAliveConnection zkw = ((ConnectionImplementation) connection).
              getKeepAliveZooKeeperWatcher();) {
+
       // This is NASTY. FIX!!!! Dependent on internal implementation! TODO
       zkw.getRecoverableZooKeeper().getZooKeeper().exists(zkw.baseZNode, false);
       connection.isMasterRunning();
@@ -2135,15 +2231,14 @@ public class HBaseAdmin implements Admin {
   @Override
   public HTableDescriptor[] getTableDescriptorsByTableName(final List<TableName> tableNames)
   throws IOException {
-    return executeCallable(new MasterCallable<HTableDescriptor[]>(getConnection(),
-        getRpcControllerFactory()) {
+    return executeCallable(new MasterCallable<HTableDescriptor[]>(getConnection()) {
       @Override
-      protected HTableDescriptor[] call(PayloadCarryingRpcController rpcController)
-      throws Exception {
+      public HTableDescriptor[] call(int callTimeout) throws Exception {
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
         GetTableDescriptorsRequest req =
             RequestConverter.buildGetTableDescriptorsRequest(tableNames);
-          return ProtobufUtil.
-              getHTableDescriptorArray(master.getTableDescriptors(rpcController, req));
+          return ProtobufUtil.getHTableDescriptorArray(master.getTableDescriptors(controller, req));
       }
     });
   }
@@ -2181,16 +2276,16 @@ public class HBaseAdmin implements Admin {
 
   private RollWALWriterResponse rollWALWriterImpl(final ServerName sn) throws IOException,
       FailedLogCloseException {
-    final AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
-    Callable<RollWALWriterResponse> callable = new Callable<RollWALWriterResponse>() {
-      @Override
-      public RollWALWriterResponse call() throws Exception {
-        RollWALWriterRequest request = RequestConverter.buildRollWALWriterRequest();
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        return admin.rollWALWriter(controller, request);
-      }
-    };
-    return ProtobufUtil.call(callable);
+    AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
+    RollWALWriterRequest request = RequestConverter.buildRollWALWriterRequest();
+    PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+
+    try {
+      // TODO: this does not do retries, it should. Set priority and timeout in controller
+      return admin.rollWALWriter(controller, request);
+    } catch (ServiceException se) {
+      throw ProtobufUtil.getRemoteException(se);
+    }
   }
 
   /**
@@ -2226,7 +2321,8 @@ public class HBaseAdmin implements Admin {
     }
     byte[][] regionsToFlush = new byte[regionCount][];
     for (int i = 0; i < regionCount; i++) {
-      regionsToFlush[i] = ProtobufUtil.toBytes(response.getRegionToFlush(i));
+      ByteString region = response.getRegionToFlush(i);
+      regionsToFlush[i] = region.toByteArray();
     }
     return regionsToFlush;
   }
@@ -2256,31 +2352,28 @@ public class HBaseAdmin implements Admin {
   @Override
   public CompactionState getCompactionStateForRegion(final byte[] regionName)
   throws IOException {
-    final Pair<HRegionInfo, ServerName> regionServerPair = getRegion(regionName);
-    if (regionServerPair == null) {
-      throw new IllegalArgumentException("Invalid region: " + Bytes.toStringBinary(regionName));
-    }
-    if (regionServerPair.getSecond() == null) {
-      throw new NoServerForRegionException(Bytes.toStringBinary(regionName));
-    }
-    ServerName sn = regionServerPair.getSecond();
-    final AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
-    Callable<CompactionState> callable = new Callable<CompactionState>() {
-      @Override
-      public CompactionState call() throws Exception {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        GetRegionInfoRequest request = RequestConverter.buildGetRegionInfoRequest(
-            regionServerPair.getFirst().getRegionName(), true);
-
-        // TODO: this does not do retries, it should. Set priority and timeout in controller
-        GetRegionInfoResponse response = admin.getRegionInfo(controller, request);
-        if (response.getCompactionState() != null) {
-          return ProtobufUtil.createCompactionState(response.getCompactionState());
-        }
-        return null;
+    try {
+      Pair<HRegionInfo, ServerName> regionServerPair = getRegion(regionName);
+      if (regionServerPair == null) {
+        throw new IllegalArgumentException("Invalid region: " + Bytes.toStringBinary(regionName));
+      }
+      if (regionServerPair.getSecond() == null) {
+        throw new NoServerForRegionException(Bytes.toStringBinary(regionName));
+      }
+      ServerName sn = regionServerPair.getSecond();
+      AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
+      GetRegionInfoRequest request = RequestConverter.buildGetRegionInfoRequest(
+        regionServerPair.getFirst().getRegionName(), true);
+      PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+      // TODO: this does not do retries, it should. Set priority and timeout in controller
+      GetRegionInfoResponse response = admin.getRegionInfo(controller, request);
+      if (response.getCompactionState() != null) {
+        return ProtobufUtil.createCompactionState(response.getCompactionState());
       }
-    };
-    return ProtobufUtil.call(callable);
+      return null;
+    } catch (ServiceException se) {
+      throw ProtobufUtil.getRemoteException(se);
+    }
   }
 
   @Override
@@ -2332,12 +2425,12 @@ public class HBaseAdmin implements Admin {
         throw (InterruptedIOException)new InterruptedIOException("Interrupted").initCause(e);
       }
       LOG.debug("Getting current status of snapshot from master...");
-      done = executeCallable(new MasterCallable<IsSnapshotDoneResponse>(getConnection(),
-          getRpcControllerFactory()) {
+      done = executeCallable(new MasterCallable<IsSnapshotDoneResponse>(getConnection()) {
         @Override
-        protected IsSnapshotDoneResponse call(PayloadCarryingRpcController rpcController)
-        throws Exception {
-          return master.isSnapshotDone(rpcController, request);
+        public IsSnapshotDoneResponse call(int callTimeout) throws ServiceException {
+          PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+          controller.setCallTimeout(callTimeout);
+          return master.isSnapshotDone(controller, request);
         }
       });
     }
@@ -2383,12 +2476,12 @@ public class HBaseAdmin implements Admin {
     final SnapshotRequest request = SnapshotRequest.newBuilder().setSnapshot(snapshot)
         .build();
     // run the snapshot on the master
-    return executeCallable(new MasterCallable<SnapshotResponse>(getConnection(),
-        getRpcControllerFactory()) {
+    return executeCallable(new MasterCallable<SnapshotResponse>(getConnection()) {
       @Override
-      protected SnapshotResponse call(PayloadCarryingRpcController rpcController)
-      throws Exception {
-        return master.snapshot(rpcController, request);
+      public SnapshotResponse call(int callTimeout) throws ServiceException {
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
+        return master.snapshot(controller, request);
       }
     });
   }
@@ -2397,12 +2490,12 @@ public class HBaseAdmin implements Admin {
   public boolean isSnapshotFinished(final SnapshotDescription snapshotDesc)
       throws IOException, HBaseSnapshotException, UnknownSnapshotException {
     final HBaseProtos.SnapshotDescription snapshot = createHBaseProtosSnapshotDesc(snapshotDesc);
-    return executeCallable(new MasterCallable<IsSnapshotDoneResponse>(getConnection(),
-        getRpcControllerFactory()) {
+    return executeCallable(new MasterCallable<IsSnapshotDoneResponse>(getConnection()) {
       @Override
-      protected IsSnapshotDoneResponse call(PayloadCarryingRpcController rpcController)
-      throws Exception {
-        return master.isSnapshotDone(rpcController,
+      public IsSnapshotDoneResponse call(int callTimeout) throws ServiceException {
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
+        return master.isSnapshotDone(controller,
           IsSnapshotDoneRequest.newBuilder().setSnapshot(snapshot).build());
       }
     }).getDone();
@@ -2581,11 +2674,12 @@ public class HBaseAdmin implements Admin {
         .setProcedure(builder.build()).build();
     // run the procedure on the master
     ExecProcedureResponse response = executeCallable(new MasterCallable<ExecProcedureResponse>(
-        getConnection(), getRpcControllerFactory()) {
+        getConnection()) {
       @Override
-      protected ExecProcedureResponse call(PayloadCarryingRpcController rpcController)
-      throws Exception {
-        return master.execProcedureWithRet(rpcController, request);
+      public ExecProcedureResponse call(int callTimeout) throws ServiceException {
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
+        return master.execProcedureWithRet(controller, request);
       }
     });
 
@@ -2607,11 +2701,12 @@ public class HBaseAdmin implements Admin {
         .setProcedure(builder.build()).build();
     // run the procedure on the master
     ExecProcedureResponse response = executeCallable(new MasterCallable<ExecProcedureResponse>(
-        getConnection(), getRpcControllerFactory()) {
+        getConnection()) {
       @Override
-      protected ExecProcedureResponse call(PayloadCarryingRpcController rpcController)
-      throws Exception {
-        return master.execProcedure(rpcController, request);
+      public ExecProcedureResponse call(int callTimeout) throws ServiceException {
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
+        return master.execProcedure(controller, request);
       }
     });
 
@@ -2655,11 +2750,12 @@ public class HBaseAdmin implements Admin {
     }
     final ProcedureDescription desc = builder.build();
     return executeCallable(
-        new MasterCallable<IsProcedureDoneResponse>(getConnection(), getRpcControllerFactory()) {
+        new MasterCallable<IsProcedureDoneResponse>(getConnection()) {
           @Override
-          protected IsProcedureDoneResponse call(PayloadCarryingRpcController rpcController)
-          throws Exception {
-            return master.isProcedureDone(rpcController, IsProcedureDoneRequest
+          public IsProcedureDoneResponse call(int callTimeout) throws ServiceException {
+            PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+            controller.setCallTimeout(callTimeout);
+            return master.isProcedureDone(controller, IsProcedureDoneRequest
                 .newBuilder().setProcedure(desc).build());
           }
         }).getDone();
@@ -2685,16 +2781,17 @@ public class HBaseAdmin implements Admin {
     ClientSnapshotDescriptionUtils.assertSnapshotRequestIsValid(snapshot);
 
     RestoreSnapshotResponse response = executeCallable(
-        new MasterCallable<RestoreSnapshotResponse>(getConnection(), getRpcControllerFactory()) {
+        new MasterCallable<RestoreSnapshotResponse>(getConnection()) {
       @Override
-      protected RestoreSnapshotResponse call(PayloadCarryingRpcController rpcController)
-      throws Exception {
+      public RestoreSnapshotResponse call(int callTimeout) throws ServiceException {
         final RestoreSnapshotRequest request = RestoreSnapshotRequest.newBuilder()
             .setSnapshot(snapshot)
             .setNonceGroup(ng.getNonceGroup())
             .setNonce(ng.newNonce())
             .build();
-        return master.restoreSnapshot(rpcController, request);
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
+        return master.restoreSnapshot(controller, request);
       }
     });
 
@@ -2731,13 +2828,13 @@ public class HBaseAdmin implements Admin {
 
   @Override
   public List<SnapshotDescription> listSnapshots() throws IOException {
-    return executeCallable(new MasterCallable<List<SnapshotDescription>>(getConnection(),
-        getRpcControllerFactory()) {
+    return executeCallable(new MasterCallable<List<SnapshotDescription>>(getConnection()) {
       @Override
-      protected List<SnapshotDescription> call(PayloadCarryingRpcController rpcController)
-      throws Exception {
+      public List<SnapshotDescription> call(int callTimeout) throws ServiceException {
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
         List<HBaseProtos.SnapshotDescription> snapshotsList = master
-            .getCompletedSnapshots(rpcController, GetCompletedSnapshotsRequest.newBuilder().build())
+            .getCompletedSnapshots(controller, GetCompletedSnapshotsRequest.newBuilder().build())
             .getSnapshotsList();
         List<SnapshotDescription> result = new ArrayList<SnapshotDescription>(snapshotsList.size());
         for (HBaseProtos.SnapshotDescription snapshot : snapshotsList) {
@@ -2800,11 +2897,14 @@ public class HBaseAdmin implements Admin {
     // make sure the snapshot is possibly valid
     TableName.isLegalFullyQualifiedTableName(Bytes.toBytes(snapshotName));
     // do the delete
-    executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
+    executeCallable(new MasterCallable<Void>(getConnection()) {
       @Override
-      protected Void call(PayloadCarryingRpcController rpcController) throws Exception {
-        master.deleteSnapshot(rpcController,
-          DeleteSnapshotRequest.newBuilder().setSnapshot(
+      public Void call(int callTimeout) throws ServiceException {
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
+        master.deleteSnapshot(controller,
+          DeleteSnapshotRequest.newBuilder().
+              setSnapshot(
                 HBaseProtos.SnapshotDescription.newBuilder().setName(snapshotName).build())
               .build()
         );
@@ -2833,10 +2933,12 @@ public class HBaseAdmin implements Admin {
   }
 
   private void internalDeleteSnapshot(final SnapshotDescription snapshot) throws IOException {
-    executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
+    executeCallable(new MasterCallable<Void>(getConnection()) {
       @Override
-      protected Void call(PayloadCarryingRpcController rpcController) throws Exception {
-        this.master.deleteSnapshot(rpcController, DeleteSnapshotRequest.newBuilder()
+      public Void call(int callTimeout) throws ServiceException {
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
+        this.master.deleteSnapshot(controller, DeleteSnapshotRequest.newBuilder()
           .setSnapshot(createHBaseProtosSnapshotDesc(snapshot)).build());
         return null;
       }
@@ -2865,10 +2967,11 @@ public class HBaseAdmin implements Admin {
 
   @Override
   public void setQuota(final QuotaSettings quota) throws IOException {
-    executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
+    executeCallable(new MasterCallable<Void>(getConnection()) {
       @Override
-      protected Void call(PayloadCarryingRpcController rpcController) throws Exception {
+      public Void call(int callTimeout) throws ServiceException {
         PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
         this.master.setQuota(controller, QuotaSettings.buildSetQuotaRequestProto(quota));
         return null;
       }
@@ -2886,8 +2989,8 @@ public class HBaseAdmin implements Admin {
   }
 
   static private <C extends RetryingCallable<V> & Closeable, V> V executeCallable(C callable,
-             RpcRetryingCallerFactory rpcCallerFactory, int operationTimeout, int rpcTimeout)
-  throws IOException {
+             RpcRetryingCallerFactory rpcCallerFactory, int operationTimeout,
+      int rpcTimeout) throws IOException {
     RpcRetryingCaller<V> caller = rpcCallerFactory.newCaller(rpcTimeout);
     try {
       return caller.callWithRetries(callable, operationTimeout);
@@ -2905,6 +3008,7 @@ public class HBaseAdmin implements Admin {
    * Simple {@link Abortable}, throwing RuntimeException on abort.
    */
   private static class ThrowableAbortable implements Abortable {
+
     @Override
     public void abort(String why, Throwable e) {
       throw new RuntimeException(why, e);
@@ -2922,16 +3026,13 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public void updateConfiguration(final ServerName server) throws IOException {
-    final AdminService.BlockingInterface admin = this.connection.getAdmin(server);
-    Callable<Void> callable = new Callable<Void>() {
-      @Override
-      public Void call() throws Exception {
-        admin.updateConfiguration(null, UpdateConfigurationRequest.getDefaultInstance());
-        return null;
-      }
-    };
-    ProtobufUtil.call(callable);
+  public void updateConfiguration(ServerName server) throws IOException {
+    try {
+      this.connection.getAdmin(server).updateConfiguration(null,
+        UpdateConfigurationRequest.getDefaultInstance());
+    } catch (ServiceException e) {
+      throw ProtobufUtil.getRemoteException(e);
+    }
   }
 
   @Override
@@ -2944,7 +3045,8 @@ public class HBaseAdmin implements Admin {
   @Override
   public int getMasterInfoPort() throws IOException {
     // TODO: Fix!  Reaching into internal implementation!!!!
-    ConnectionImplementation connection = (ConnectionImplementation)this.connection;
+    ConnectionImplementation connection =
+        (ConnectionImplementation)this.connection;
     ZooKeeperKeepAliveConnection zkw = connection.getKeepAliveZooKeeperWatcher();
     try {
       return MasterAddressTracker.getMasterInfoPort(zkw);
@@ -2955,7 +3057,8 @@ public class HBaseAdmin implements Admin {
 
   private ServerName getMasterAddress() throws IOException {
     // TODO: Fix!  Reaching into internal implementation!!!!
-    ConnectionImplementation connection = (ConnectionImplementation)this.connection;
+    ConnectionImplementation connection =
+            (ConnectionImplementation)this.connection;
     ZooKeeperKeepAliveConnection zkw = connection.getKeepAliveZooKeeperWatcher();
     try {
       return MasterAddressTracker.getMasterAddress(zkw);
@@ -2966,26 +3069,33 @@ public class HBaseAdmin implements Admin {
 
   @Override
   public long getLastMajorCompactionTimestamp(final TableName tableName) throws IOException {
-    return executeCallable(new MasterCallable<Long>(getConnection(), getRpcControllerFactory()) {
+    return executeCallable(new MasterCallable<Long>(getConnection()) {
       @Override
-      protected Long call(PayloadCarryingRpcController rpcController) throws Exception {
+      public Long call(int callTimeout) throws ServiceException {
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
         MajorCompactionTimestampRequest req =
             MajorCompactionTimestampRequest.newBuilder()
                 .setTableName(ProtobufUtil.toProtoTableName(tableName)).build();
-        return master.getLastMajorCompactionTimestamp(rpcController, req).getCompactionTimestamp();
+        return master.getLastMajorCompactionTimestamp(controller, req).getCompactionTimestamp();
       }
     });
   }
 
   @Override
   public long getLastMajorCompactionTimestampForRegion(final byte[] regionName) throws IOException {
-    return executeCallable(new MasterCallable<Long>(getConnection(), getRpcControllerFactory()) {
+    return executeCallable(new MasterCallable<Long>(getConnection()) {
       @Override
-      protected Long call(PayloadCarryingRpcController rpcController) throws Exception {
+      public Long call(int callTimeout) throws ServiceException {
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        controller.setCallTimeout(callTimeout);
         MajorCompactionTimestampForRegionRequest req =
-            MajorCompactionTimestampForRegionRequest.newBuilder().setRegion(RequestConverter
+            MajorCompactionTimestampForRegionRequest
+                .newBuilder()
+                .setRegion(
+                  RequestConverter
                       .buildRegionSpecifier(RegionSpecifierType.REGION_NAME, regionName)).build();
-        return master.getLastMajorCompactionTimestampForRegion(rpcController, req)
+        return master.getLastMajorCompactionTimestampForRegion(controller, req)
             .getCompactionTimestamp();
       }
     });
@@ -3024,35 +3134,32 @@ public class HBaseAdmin implements Admin {
   @Override
   public void majorCompact(final TableName tableName, CompactType compactType)
           throws IOException, InterruptedException {
-    compact(tableName, null, true, compactType);
+      compact(tableName, null, true, compactType);
   }
 
   /**
    * {@inheritDoc}
    */
   @Override
-  public CompactionState getCompactionState(final TableName tableName,
+  public CompactionState getCompactionState(TableName tableName,
     CompactType compactType) throws IOException {
     AdminProtos.GetRegionInfoResponse.CompactionState state =
         AdminProtos.GetRegionInfoResponse.CompactionState.NONE;
     checkTableExists(tableName);
-    final PayloadCarryingRpcController rpcController = rpcControllerFactory.newController();
+    PayloadCarryingRpcController controller = rpcControllerFactory.newController();
     switch (compactType) {
       case MOB:
-        final AdminProtos.AdminService.BlockingInterface masterAdmin =
-          this.connection.getAdmin(getMasterAddress());
-        Callable<AdminProtos.GetRegionInfoResponse.CompactionState> callable =
-            new Callable<AdminProtos.GetRegionInfoResponse.CompactionState>() {
-          @Override
-          public AdminProtos.GetRegionInfoResponse.CompactionState call() throws Exception {
-            HRegionInfo info = getMobRegionInfo(tableName);
-            GetRegionInfoRequest request = RequestConverter.buildGetRegionInfoRequest(
-                info.getRegionName(), true);
-            GetRegionInfoResponse response = masterAdmin.getRegionInfo(rpcController, request);
-            return response.getCompactionState();
-          }
-        };
-        state = ProtobufUtil.call(callable);
+        try {
+          ServerName master = getMasterAddress();
+          HRegionInfo info = getMobRegionInfo(tableName);
+          GetRegionInfoRequest request = RequestConverter.buildGetRegionInfoRequest(
+                  info.getRegionName(), true);
+          GetRegionInfoResponse response = this.connection.getAdmin(master)
+                  .getRegionInfo(controller, request);
+          state = response.getCompactionState();
+        } catch (ServiceException se) {
+          throw ProtobufUtil.getRemoteException(se);
+        }
         break;
       case NORMAL:
       default:
@@ -3066,23 +3173,15 @@ public class HBaseAdmin implements Admin {
           } else {
             pairs = MetaTableAccessor.getTableRegionsAndLocations(connection, tableName);
           }
-          for (Pair<HRegionInfo, ServerName> pair: pairs) {
+          for (Pair<HRegionInfo, ServerName> pair : pairs) {
             if (pair.getFirst().isOffline()) continue;
             if (pair.getSecond() == null) continue;
-            final ServerName sn = pair.getSecond();
-            final byte [] regionName = pair.getFirst().getRegionName();
-            final AdminService.BlockingInterface snAdmin = this.connection.getAdmin(sn);
             try {
-              Callable<GetRegionInfoResponse> regionInfoCallable =
-                  new Callable<GetRegionInfoResponse>() {
-                @Override
-                public GetRegionInfoResponse call() throws Exception {
-                  GetRegionInfoRequest request = RequestConverter.buildGetRegionInfoRequest(
-                      regionName, true);
-                  return snAdmin.getRegionInfo(rpcController, request);
-                }
-              };
-              GetRegionInfoResponse response = ProtobufUtil.call(regionInfoCallable);
+              ServerName sn = pair.getSecond();
+              AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
+              GetRegionInfoRequest request = RequestConverter.buildGetRegionInfoRequest(
+                      pair.getFirst().getRegionName(), true);
+              GetRegionInfoResponse response = admin.getRegionInfo(controller, request);
               switch (response.getCompactionState()) {
                 case MAJOR_AND_MINOR:
                   return CompactionState.MAJOR_AND_MINOR;
@@ -3118,6 +3217,8 @@ public class HBaseAdmin implements Admin {
               }
             }
           }
+        } catch (ServiceException se) {
+          throw ProtobufUtil.getRemoteException(se);
         } finally {
           if (zookeeper != null) {
             zookeeper.close();
@@ -3182,11 +3283,12 @@ public class HBaseAdmin implements Admin {
     protected AbortProcedureResponse abortProcedureResult(
         final AbortProcedureRequest request) throws IOException {
       return admin.executeCallable(new MasterCallable<AbortProcedureResponse>(
-          admin.getConnection(), admin.getRpcControllerFactory()) {
+          admin.getConnection()) {
         @Override
-        protected AbortProcedureResponse call(PayloadCarryingRpcController rcpController)
-        throws Exception {
-          return master.abortProcedure(rcpController, request);
+        public AbortProcedureResponse call(int callTimeout) throws ServiceException {
+          PayloadCarryingRpcController controller = admin.getRpcControllerFactory().newCon

<TRUNCATED>

[48/50] [abbrv] hbase git commit: HBASE-16341 Missing bit on "Regression: Random Read/WorkloadC slower in 1.x than 0.98"

Posted by sy...@apache.org.
HBASE-16341 Missing bit on "Regression: Random Read/WorkloadC slower in 1.x than 0.98"


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

Branch: refs/heads/hbase-12439
Commit: 907ac1f6552e9e1cd7e33adeaf51fda2f3f19734
Parents: 65c03b0
Author: stack <st...@apache.org>
Authored: Sat Aug 13 20:16:44 2016 -0700
Committer: stack <st...@apache.org>
Committed: Sat Aug 13 20:16:44 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/ipc/SimpleRpcScheduler.java  | 7 ++-----
 1 file changed, 2 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/907ac1f6/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcScheduler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcScheduler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcScheduler.java
index 082c743..3ea2887 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcScheduler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcScheduler.java
@@ -96,9 +96,7 @@ public class SimpleRpcScheduler extends RpcScheduler implements ConfigurationObs
       replicationExecutor.resizeQueues(conf);
     }
 
-    String callQueueType = conf.get(CALL_QUEUE_TYPE_CONF_KEY,
-      CALL_QUEUE_TYPE_DEADLINE_CONF_VALUE);
-
+    String callQueueType = conf.get(CALL_QUEUE_TYPE_CONF_KEY, CALL_QUEUE_TYPE_CONF_DEFAULT);
     if (isCodelQueueType(callQueueType)) {
       // update CoDel Scheduler tunables
       int codelTargetDelay = conf.getInt(CALL_QUEUE_CODEL_TARGET_DELAY,
@@ -184,8 +182,7 @@ public class SimpleRpcScheduler extends RpcScheduler implements ConfigurationObs
     this.highPriorityLevel = highPriorityLevel;
     this.abortable = server;
 
-    String callQueueType = conf.get(CALL_QUEUE_TYPE_CONF_KEY,
-        CALL_QUEUE_TYPE_FIFO_CONF_VALUE);
+    String callQueueType = conf.get(CALL_QUEUE_TYPE_CONF_KEY, CALL_QUEUE_TYPE_CONF_DEFAULT);
     float callqReadShare = conf.getFloat(CALL_QUEUE_READ_SHARE_CONF_KEY, 0);
     float callqScanShare = conf.getFloat(CALL_QUEUE_SCAN_SHARE_CONF_KEY, 0);
 


[08/50] [abbrv] hbase git commit: HBASE-16303 Addendum as the wrong patch was committed

Posted by sy...@apache.org.
HBASE-16303 Addendum as the wrong patch was committed


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

Branch: refs/heads/hbase-12439
Commit: e8f08ba0c7949edb4bc349718d2d6532be2f2360
Parents: 19c609f
Author: Ramkrishna <ra...@intel.com>
Authored: Fri Aug 5 13:06:46 2016 +0530
Committer: Ramkrishna <ra...@intel.com>
Committed: Fri Aug 5 13:06:46 2016 +0530

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/hbase/filter/FilterList.java    | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/e8f08ba0/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java
index da7a084..1ea3c80 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java
@@ -285,6 +285,7 @@ final public class FilterList extends Filter {
         }
       } else if (operator == Operator.MUST_PASS_ONE) {
         if (filter.filterAllRemaining()) {
+          seenNonHintReturnCode = true;
           continue;
         }
 


[21/50] [abbrv] hbase git commit: Revert "HBASE-16308 Contain protobuf references Gather up the pb references into a few locations only rather than have pb references distributed all about the code base."

Posted by sy...@apache.org.
Revert "HBASE-16308 Contain protobuf references Gather up the pb references into a few locations only rather than have pb references distributed all about the code base."

This reverts commit ed87a81b4b61c4842c12572a47c97ae23773012f.


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

Branch: refs/heads/hbase-12439
Commit: 0206dc67d643e4a248a319c724cd6e58f0e77603
Parents: ed87a81
Author: stack <st...@apache.org>
Authored: Fri Aug 5 15:18:48 2016 -0700
Committer: stack <st...@apache.org>
Committed: Fri Aug 5 15:18:48 2016 -0700

----------------------------------------------------------------------
 .../client/AbstractRegionServerCallable.java    |   23 +-
 .../hadoop/hbase/client/ClientScanner.java      |    2 +-
 .../hbase/client/ClientSimpleScanner.java       |    3 +-
 .../hadoop/hbase/client/ClientSmallScanner.java |   42 +-
 .../hadoop/hbase/client/ConnectionCallable.java |   56 +
 .../hbase/client/ConnectionImplementation.java  |   40 +-
 .../hbase/client/FlushRegionCallable.java       |   26 +-
 .../apache/hadoop/hbase/client/HBaseAdmin.java  | 1110 ++++++++++--------
 .../org/apache/hadoop/hbase/client/HTable.java  |  455 ++++---
 .../hadoop/hbase/client/MasterCallable.java     |   37 +-
 .../hbase/client/MasterKeepAliveConnection.java |    3 +-
 .../hbase/client/MultiServerCallable.java       |   35 +-
 .../client/PayloadCarryingServerCallable.java   |   44 +-
 .../client/RegionAdminServiceCallable.java      |   54 +-
 .../hbase/client/RegionServerCallable.java      |   72 +-
 .../hbase/client/RetryingTimeTracker.java       |   12 +-
 .../hbase/client/ReversedScannerCallable.java   |    4 +-
 .../hbase/client/RpcRetryingCallable.java       |   65 -
 .../hadoop/hbase/client/RpcRetryingCaller.java  |    5 +-
 .../hbase/client/RpcRetryingCallerFactory.java  |    1 -
 .../RpcRetryingCallerWithReadReplicas.java      |   26 +-
 .../hadoop/hbase/client/ScannerCallable.java    |  140 ++-
 .../hbase/client/SecureBulkLoadClient.java      |   80 +-
 .../hbase/ipc/MasterCoprocessorRpcChannel.java  |    3 +-
 .../hbase/ipc/PayloadCarryingRpcController.java |  139 +--
 .../hbase/ipc/RegionCoprocessorRpcChannel.java  |   23 +-
 .../hbase/ipc/TimeLimitedRpcController.java     |  142 +++
 .../hadoop/hbase/protobuf/ProtobufUtil.java     |   73 +-
 .../hadoop/hbase/client/TestClientScanner.java  |    1 +
 .../apache/hadoop/hbase/HBaseIOException.java   |    3 +-
 .../apache/hadoop/hbase/util/ExceptionUtil.java |    2 +-
 .../org/apache/hadoop/hbase/ipc/RpcServer.java  |    6 +-
 .../hbase/mapreduce/LoadIncrementalHFiles.java  |   36 +-
 .../master/ExpiredMobFileCleanerChore.java      |    6 +
 .../hadoop/hbase/master/ServerManager.java      |    5 +-
 .../hadoop/hbase/mob/ExpiredMobFileCleaner.java |   12 +-
 .../hadoop/hbase/mob/mapreduce/Sweeper.java     |    6 +-
 .../hbase/regionserver/RSRpcServices.java       |   15 +-
 .../regionserver/wal/WALEditsReplaySink.java    |   43 +-
 .../RegionReplicaReplicationEndpoint.java       |   54 +-
 .../org/apache/hadoop/hbase/util/Merge.java     |   13 +-
 .../org/apache/hadoop/hbase/TestNamespace.java  |    7 +-
 .../apache/hadoop/hbase/client/TestAdmin2.java  |    8 +-
 .../hadoop/hbase/client/TestClientTimeouts.java |    7 +-
 .../org/apache/hadoop/hbase/client/TestHCM.java |   37 +-
 .../hbase/client/TestReplicaWithCluster.java    |   35 +-
 .../regionserver/TestHRegionServerBulkLoad.java |   23 +-
 .../TestHRegionServerBulkLoadWithOldClient.java |   13 +-
 ...gionServerBulkLoadWithOldSecureEndpoint.java |   27 +-
 .../hbase/spark/SparkSQLPushDownFilter.java     |    4 +-
 50 files changed, 1630 insertions(+), 1448 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/0206dc67/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AbstractRegionServerCallable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AbstractRegionServerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AbstractRegionServerCallable.java
index 5a1f5cc..7279d81 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AbstractRegionServerCallable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AbstractRegionServerCallable.java
@@ -18,7 +18,8 @@
 package org.apache.hadoop.hbase.client;
 
 import java.io.IOException;
-
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.ServerName;
@@ -28,15 +29,26 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.util.Bytes;
 
 /**
- * Added by HBASE-15745 Refactor of RPC classes to better accept async changes.
- * Temporary.
+ * Implementations call a RegionServer.
+ * Passed to a {@link RpcRetryingCaller} so we retry on fail.
+ * TODO: this class is actually tied to one region, because most of the paths make use of
+ *       the regioninfo part of location when building requests. The only reason it works for
+ *       multi-region requests (e.g. batch) is that they happen to not use the region parts.
+ *       This could be done cleaner (e.g. having a generic parameter and 2 derived classes,
+ *       RegionCallable and actual RegionServerCallable with ServerName.
+ * @param <T> the class that the ServerCallable handles
  */
 @InterfaceAudience.Private
 abstract class AbstractRegionServerCallable<T> implements RetryingCallable<T> {
+  // Public because used outside of this package over in ipc.
+  private static final Log LOG = LogFactory.getLog(AbstractRegionServerCallable.class);
+
   protected final Connection connection;
   protected final TableName tableName;
   protected final byte[] row;
+
   protected HRegionLocation location;
+
   protected final static int MIN_WAIT_DEAD_SERVER = 10000;
 
   /**
@@ -115,7 +127,8 @@ abstract class AbstractRegionServerCallable<T> implements RetryingCallable<T> {
   @Override
   public void prepare(final boolean reload) throws IOException {
     // check table state if this is a retry
-    if (reload && !tableName.equals(TableName.META_TABLE_NAME) &&
+    if (reload &&
+        !tableName.equals(TableName.META_TABLE_NAME) &&
         getConnection().isTableDisabled(tableName)) {
       throw new TableNotEnabledException(tableName.getNameAsString() + " is disabled.");
     }
@@ -135,4 +148,4 @@ abstract class AbstractRegionServerCallable<T> implements RetryingCallable<T> {
    * @throws IOException When client could not be created
    */
   abstract void setClientByServiceName(ServerName serviceName) throws IOException;
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0206dc67/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java
index 3e676c7..cb4c714 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java
@@ -847,4 +847,4 @@ public abstract class ClientScanner extends AbstractClientScanner {
     Cell[] list = Arrays.copyOfRange(result.rawCells(), index, result.rawCells().length);
     return Result.create(list, result.getExists(), result.isStale(), result.isPartial());
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0206dc67/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSimpleScanner.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSimpleScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSimpleScanner.java
index ecf083b..f886971 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSimpleScanner.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSimpleScanner.java
@@ -32,6 +32,7 @@ import java.util.concurrent.ExecutorService;
  */
 @InterfaceAudience.Private
 public class ClientSimpleScanner extends ClientScanner {
+
   public ClientSimpleScanner(Configuration configuration, Scan scan, TableName name,
       ClusterConnection connection, RpcRetryingCallerFactory rpcCallerFactory,
       RpcControllerFactory rpcControllerFactory, ExecutorService pool,
@@ -49,4 +50,4 @@ public class ClientSimpleScanner extends ClientScanner {
   public Result next() throws IOException {
     return nextWithSyncCache();
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0206dc67/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallScanner.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallScanner.java
index 429c4cf..f9bdd55 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallScanner.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallScanner.java
@@ -18,10 +18,8 @@
  */
 package org.apache.hadoop.hbase.client;
 
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.util.concurrent.ExecutorService;
-
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.ServiceException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -31,15 +29,17 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
-import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.protobuf.ResponseConverter;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse;
 import org.apache.hadoop.hbase.util.Bytes;
 
-import com.google.common.annotations.VisibleForTesting;
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.util.concurrent.ExecutorService;
 
 /**
  * Client scanner for small scan. Generally, only one RPC is called to fetch the
@@ -185,7 +185,7 @@ public class ClientSmallScanner extends ClientSimpleScanner {
     }
 
     @Override
-    protected Result[] call(PayloadCarryingRpcController controller) throws Exception {
+    public Result[] call(int timeout) throws IOException {
       if (this.closed) return null;
       if (Thread.interrupted()) {
         throw new InterruptedIOException();
@@ -193,17 +193,25 @@ public class ClientSmallScanner extends ClientSimpleScanner {
       ScanRequest request = RequestConverter.buildScanRequest(getLocation()
           .getRegionInfo().getRegionName(), getScan(), getCaching(), true);
       ScanResponse response = null;
-      response = getStub().scan(controller, request);
-      Result[] results = ResponseConverter.getResults(controller.cellScanner(), response);
-      if (response.hasMoreResultsInRegion()) {
-        setHasMoreResultsContext(true);
-        setServerHasMoreResults(response.getMoreResultsInRegion());
-      } else {
-        setHasMoreResultsContext(false);
+      controller = controllerFactory.newController();
+      try {
+        controller.setPriority(getTableName());
+        controller.setCallTimeout(timeout);
+        response = getStub().scan(controller, request);
+        Result[] results = ResponseConverter.getResults(controller.cellScanner(),
+            response);
+        if (response.hasMoreResultsInRegion()) {
+          setHasMoreResultsContext(true);
+          setServerHasMoreResults(response.getMoreResultsInRegion());
+        } else {
+          setHasMoreResultsContext(false);
+        }
+        // We need to update result metrics since we are overriding call()
+        updateResultsMetrics(results);
+        return results;
+      } catch (ServiceException se) {
+        throw ProtobufUtil.getRemoteException(se);
       }
-      // We need to update result metrics since we are overriding call()
-      updateResultsMetrics(results);
-      return results;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/0206dc67/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionCallable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionCallable.java
new file mode 100644
index 0000000..3f44927
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionCallable.java
@@ -0,0 +1,56 @@
+/**
+ * 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
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.client;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+/**
+ * A RetryingCallable for generic connection operations.
+ * @param <V> return type
+ */
+abstract class ConnectionCallable<V> implements RetryingCallable<V>, Closeable {
+  protected Connection connection;
+
+  public ConnectionCallable(final Connection connection) {
+    this.connection = connection;
+  }
+
+  @Override
+  public void prepare(boolean reload) throws IOException {
+  }
+
+  @Override
+  public void close() throws IOException {
+  }
+
+  @Override
+  public void throwable(Throwable t, boolean retrying) {
+  }
+
+  @Override
+  public String getExceptionMessageAdditionalDetail() {
+    return "";
+  }
+
+  @Override
+  public long sleep(long pause, int tries) {
+    return ConnectionUtils.getPauseTime(pause, tries);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/0206dc67/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
index 638050f..8dcda13 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
@@ -20,6 +20,11 @@ package org.apache.hadoop.hbase.client;
 
 import static org.apache.hadoop.hbase.client.MetricsConnection.CLIENT_SIDE_METRICS_ENABLED_KEY;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.BlockingRpcChannel;
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+
 import java.io.Closeable;
 import java.io.IOException;
 import java.io.InterruptedIOException;
@@ -63,7 +68,6 @@ import org.apache.hadoop.hbase.exceptions.RegionMovedException;
 import org.apache.hadoop.hbase.ipc.RpcClient;
 import org.apache.hadoop.hbase.ipc.RpcClientFactory;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
-import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
@@ -91,11 +95,6 @@ import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.zookeeper.KeeperException;
 
-import com.google.common.annotations.VisibleForTesting;
-import com.google.protobuf.BlockingRpcChannel;
-import com.google.protobuf.RpcController;
-import com.google.protobuf.ServiceException;
-
 /**
  * Main implementation of {@link Connection} and {@link ClusterConnection} interfaces.
  * Encapsulates connection to zookeeper and regionservers.
@@ -934,13 +933,9 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
       this.stub = null;
     }
 
-    boolean isMasterRunning() throws IOException {
-      MasterProtos.IsMasterRunningResponse response = null;
-      try {
-        response = this.stub.isMasterRunning(null, RequestConverter.buildIsMasterRunningRequest());
-      } catch (Exception e) {
-        throw ProtobufUtil.handleRemoteException(e);
-      }
+    boolean isMasterRunning() throws ServiceException {
+      MasterProtos.IsMasterRunningResponse response =
+        this.stub.isMasterRunning(null, RequestConverter.buildIsMasterRunningRequest());
       return response != null? response.getIsMasterRunning(): false;
     }
   }
@@ -1063,14 +1058,14 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
     /**
      * Once setup, check it works by doing isMasterRunning check.
      */
-    protected abstract void isMasterRunning() throws IOException;
+    protected abstract void isMasterRunning() throws ServiceException;
 
     /**
      * Create a stub. Try once only.  It is not typed because there is no common type to
      * protobuf services nor their interfaces.  Let the caller do appropriate casting.
      * @return A stub for master services.
      */
-    private Object makeStubNoRetries() throws IOException, KeeperException {
+    private Object makeStubNoRetries() throws IOException, KeeperException, ServiceException {
       ZooKeeperKeepAliveConnection zkw;
       try {
         zkw = getKeepAliveZooKeeperWatcher();
@@ -1110,7 +1105,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
     }
 
     /**
-     * Create a stub against the master. Retry if necessary.
+     * Create a stub against the master.  Retry if necessary.
      * @return A stub to do <code>intf</code> against the master
      * @throws org.apache.hadoop.hbase.MasterNotRunningException if master is not running
      */
@@ -1126,7 +1121,10 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
             exceptionCaught = e;
           } catch (KeeperException e) {
             exceptionCaught = e;
+          } catch (ServiceException e) {
+            exceptionCaught = e;
           }
+
           throw new MasterNotRunningException(exceptionCaught);
         } else {
           throw new DoNotRetryIOException("Connection was closed while trying to get master");
@@ -1157,12 +1155,8 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
     }
 
     @Override
-    protected void isMasterRunning() throws IOException {
-      try {
-        this.stub.isMasterRunning(null, RequestConverter.buildIsMasterRunningRequest());
-      } catch (Exception e) {
-        throw ProtobufUtil.handleRemoteException(e);
-      }
+    protected void isMasterRunning() throws ServiceException {
+      this.stub.isMasterRunning(null, RequestConverter.buildIsMasterRunningRequest());
     }
   }
 
@@ -1707,7 +1701,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
       //  java.net.ConnectException but they're not declared. So we catch it...
       LOG.info("Master connection is not running anymore", e.getUndeclaredThrowable());
       return false;
-    } catch (IOException se) {
+    } catch (ServiceException se) {
       LOG.warn("Checking master connection", se);
       return false;
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/0206dc67/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FlushRegionCallable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FlushRegionCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FlushRegionCallable.java
index c7bf804..73bdb74 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FlushRegionCallable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FlushRegionCallable.java
@@ -27,18 +27,23 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.FlushRegionRequest;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.FlushRegionResponse;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 
+import com.google.protobuf.ServiceException;
+
 /**
  * A Callable for flushRegion() RPC.
  */
 @InterfaceAudience.Private
 public class FlushRegionCallable extends RegionAdminServiceCallable<FlushRegionResponse> {
+
   private static final Log LOG = LogFactory.getLog(FlushRegionCallable.class);
+
   private final byte[] regionName;
   private final boolean writeFlushWalMarker;
   private boolean reload;
@@ -59,14 +64,18 @@ public class FlushRegionCallable extends RegionAdminServiceCallable<FlushRegionR
   }
 
   @Override
+  public FlushRegionResponse call(int callTimeout) throws Exception {
+    return flushRegion();
+  }
+
+  @Override
   public void prepare(boolean reload) throws IOException {
     super.prepare(reload);
     this.reload = reload;
   }
 
-  @Override
-  protected FlushRegionResponse call(PayloadCarryingRpcController controller) throws Exception {
-    // Check whether we should still do the flush to this region. If the regions are changed due
+  private FlushRegionResponse flushRegion() throws IOException {
+    // check whether we should still do the flush to this region. If the regions are changed due
     // to splits or merges, etc return success
     if (!Bytes.equals(location.getRegionInfo().getRegionName(), regionName)) {
       if (!reload) {
@@ -84,6 +93,13 @@ public class FlushRegionCallable extends RegionAdminServiceCallable<FlushRegionR
 
     FlushRegionRequest request =
         RequestConverter.buildFlushRegionRequest(regionName, writeFlushWalMarker);
-    return stub.flushRegion(controller, request);
+
+    try {
+      PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+      controller.setPriority(tableName);
+      return stub.flushRegion(controller, request);
+    } catch (ServiceException se) {
+      throw ProtobufUtil.getRemoteException(se);
+    }
   }
-}
\ No newline at end of file
+}


[15/50] [abbrv] hbase git commit: HBASE-16308 Contain protobuf references Gather up the pb references into a few locations only rather than have pb references distributed all about the code base.

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/ed87a81b/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 fbd9f51..1b3e111 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
@@ -18,12 +18,6 @@
  */
 package org.apache.hadoop.hbase.client;
 
-import com.google.common.annotations.VisibleForTesting;
-import com.google.protobuf.Descriptors;
-import com.google.protobuf.Message;
-import com.google.protobuf.Service;
-import com.google.protobuf.ServiceException;
-
 import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.util.ArrayList;
@@ -43,7 +37,6 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.HTableDescriptor;
@@ -74,6 +67,16 @@ import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.ReflectionUtils;
 import org.apache.hadoop.hbase.util.Threads;
 
+import com.google.common.annotations.VisibleForTesting;
+
+// DO NOT MAKE USE OF THESE IMPORTS! THEY ARE HERE FOR COPROCESSOR ENDPOINTS ONLY.
+// Internally, we use shaded protobuf. This below are part of our public API.
+import com.google.protobuf.Descriptors;
+import com.google.protobuf.Message;
+import com.google.protobuf.ServiceException;
+import com.google.protobuf.Service;
+// SEE ABOVE NOTE!
+
 /**
  * An implementation of {@link Table}. Used to communicate with a single HBase table.
  * Lightweight. Get as needed and just close when done.
@@ -411,23 +414,16 @@ public class HTable implements Table {
 
     if (get.getConsistency() == Consistency.STRONG) {
       // Good old call.
-      final Get getReq = get;
+      final Get configuredGet = get;
       RegionServerCallable<Result> callable = new RegionServerCallable<Result>(this.connection,
-          getName(), get.getRow()) {
+          this.rpcControllerFactory, getName(), get.getRow()) {
         @Override
-        public Result call(int callTimeout) throws IOException {
-          ClientProtos.GetRequest request =
-            RequestConverter.buildGetRequest(getLocation().getRegionInfo().getRegionName(), getReq);
-          PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-          controller.setPriority(tableName);
-          controller.setCallTimeout(callTimeout);
-          try {
-            ClientProtos.GetResponse response = getStub().get(controller, request);
-            if (response == null) return null;
-            return ProtobufUtil.toResult(response.getResult(), controller.cellScanner());
-          } catch (ServiceException se) {
-            throw ProtobufUtil.getRemoteException(se);
-          }
+        protected Result call(PayloadCarryingRpcController controller) throws Exception {
+          ClientProtos.GetRequest request = RequestConverter.buildGetRequest(
+              getLocation().getRegionInfo().getRegionName(), configuredGet);
+          ClientProtos.GetResponse response = getStub().get(controller, request);
+          if (response == null) return null;
+          return ProtobufUtil.toResult(response.getResult(), controller.cellScanner());
         }
       };
       return rpcCallerFactory.<Result>newCaller(rpcTimeout).callWithRetries(callable,
@@ -443,7 +439,6 @@ public class HTable implements Table {
     return callable.call(operationTimeout);
   }
 
-
   /**
    * {@inheritDoc}
    */
@@ -454,16 +449,14 @@ public class HTable implements Table {
     }
     try {
       Object[] r1 = new Object[gets.size()];
-      batch((List) gets, r1);
-
-      // translate.
+      batch((List<? extends Row>)gets, r1);
+      // Translate.
       Result [] results = new Result[r1.length];
-      int i=0;
-      for (Object o : r1) {
-        // batch ensures if there is a failure we get an exception instead
-        results[i++] = (Result) o;
+      int i = 0;
+      for (Object obj: r1) {
+        // Batch ensures if there is a failure we get an exception instead
+        results[i++] = (Result)obj;
       }
-
       return results;
     } catch (InterruptedException e) {
       throw (InterruptedIOException)new InterruptedIOException().initCause(e);
@@ -511,21 +504,13 @@ public class HTable implements Table {
   public void delete(final Delete delete)
   throws IOException {
     RegionServerCallable<Boolean> callable = new RegionServerCallable<Boolean>(connection,
-        tableName, delete.getRow()) {
+        this.rpcControllerFactory, getName(), delete.getRow()) {
       @Override
-      public Boolean call(int callTimeout) throws IOException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setPriority(tableName);
-        controller.setCallTimeout(callTimeout);
-
-        try {
-          MutateRequest request = RequestConverter.buildMutateRequest(
-            getLocation().getRegionInfo().getRegionName(), delete);
-          MutateResponse response = getStub().mutate(controller, request);
-          return Boolean.valueOf(response.getProcessed());
-        } catch (ServiceException se) {
-          throw ProtobufUtil.getRemoteException(se);
-        }
+      protected Boolean call(PayloadCarryingRpcController controller) throws Exception {
+        MutateRequest request = RequestConverter.buildMutateRequest(
+          getLocation().getRegionInfo().getRegionName(), delete);
+        MutateResponse response = getStub().mutate(controller, request);
+        return Boolean.valueOf(response.getProcessed());
       }
     };
     rpcCallerFactory.<Boolean> newCaller(rpcTimeout).callWithRetries(callable,
@@ -581,41 +566,28 @@ public class HTable implements Table {
    */
   @Override
   public void mutateRow(final RowMutations rm) throws IOException {
-    final RetryingTimeTracker tracker = new RetryingTimeTracker();
     PayloadCarryingServerCallable<MultiResponse> callable =
-      new PayloadCarryingServerCallable<MultiResponse>(connection, getName(), rm.getRow(),
+      new PayloadCarryingServerCallable<MultiResponse>(this.connection, getName(), rm.getRow(),
           rpcControllerFactory) {
-        @Override
-        public MultiResponse call(int callTimeout) throws IOException {
-          tracker.start();
-          controller.setPriority(tableName);
-          int remainingTime = tracker.getRemainingTime(callTimeout);
-          if (remainingTime == 0) {
-            throw new DoNotRetryIOException("Timeout for mutate row");
-          }
-          controller.setCallTimeout(remainingTime);
-          try {
-            RegionAction.Builder regionMutationBuilder = RequestConverter.buildRegionAction(
-                getLocation().getRegionInfo().getRegionName(), rm);
-            regionMutationBuilder.setAtomic(true);
-            MultiRequest request =
-                MultiRequest.newBuilder().addRegionAction(regionMutationBuilder.build()).build();
-            ClientProtos.MultiResponse response = getStub().multi(controller, request);
-            ClientProtos.RegionActionResult res = response.getRegionActionResultList().get(0);
-            if (res.hasException()) {
-              Throwable ex = ProtobufUtil.toException(res.getException());
-              if (ex instanceof IOException) {
-                throw (IOException) ex;
-              }
-              throw new IOException("Failed to mutate row: " +
-                  Bytes.toStringBinary(rm.getRow()), ex);
-            }
-            return ResponseConverter.getResults(request, response, controller.cellScanner());
-          } catch (ServiceException se) {
-            throw ProtobufUtil.getRemoteException(se);
+      @Override
+      protected MultiResponse call(PayloadCarryingRpcController controller) throws Exception {
+        RegionAction.Builder regionMutationBuilder = RequestConverter.buildRegionAction(
+            getLocation().getRegionInfo().getRegionName(), rm);
+        regionMutationBuilder.setAtomic(true);
+        MultiRequest request =
+            MultiRequest.newBuilder().addRegionAction(regionMutationBuilder.build()).build();
+        ClientProtos.MultiResponse response = getStub().multi(controller, request);
+        ClientProtos.RegionActionResult res = response.getRegionActionResultList().get(0);
+        if (res.hasException()) {
+          Throwable ex = ProtobufUtil.toException(res.getException());
+          if (ex instanceof IOException) {
+            throw (IOException) ex;
           }
+          throw new IOException("Failed to mutate row: " + Bytes.toStringBinary(rm.getRow()), ex);
         }
-      };
+        return ResponseConverter.getResults(request, response, controller.cellScanner());
+      }
+    };
     AsyncRequestFuture ars = multiAp.submitAll(pool, tableName, rm.getMutations(),
         null, null, callable, operationTimeout);
     ars.waitUntilDone();
@@ -624,38 +596,31 @@ public class HTable implements Table {
     }
   }
 
+  private static void checkHasFamilies(final Mutation mutation) throws IOException {
+    if (mutation.numFamilies() == 0) {
+      throw new IOException("Invalid arguments to " + mutation + ", zero columns specified");
+    }
+  }
+
   /**
    * {@inheritDoc}
    */
   @Override
   public Result append(final Append append) throws IOException {
-    if (append.numFamilies() == 0) {
-      throw new IOException(
-          "Invalid arguments to append, no columns specified");
-    }
-
-    NonceGenerator ng = this.connection.getNonceGenerator();
-    final long nonceGroup = ng.getNonceGroup(), nonce = ng.newNonce();
-    RegionServerCallable<Result> callable =
-      new RegionServerCallable<Result>(this.connection, getName(), append.getRow()) {
-        @Override
-        public Result call(int callTimeout) throws IOException {
-          PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-          controller.setPriority(getTableName());
-          controller.setCallTimeout(callTimeout);
-          try {
-            MutateRequest request = RequestConverter.buildMutateRequest(
-              getLocation().getRegionInfo().getRegionName(), append, nonceGroup, nonce);
-            MutateResponse response = getStub().mutate(controller, request);
-            if (!response.hasResult()) return null;
-            return ProtobufUtil.toResult(response.getResult(), controller.cellScanner());
-          } catch (ServiceException se) {
-            throw ProtobufUtil.getRemoteException(se);
-          }
-        }
-      };
-    return rpcCallerFactory.<Result> newCaller(rpcTimeout).callWithRetries(callable,
-        this.operationTimeout);
+    checkHasFamilies(append);
+    RegionServerCallable<Result> callable = new RegionServerCallable<Result>(this.connection,
+        this.rpcControllerFactory, getName(), append.getRow()) {
+      @Override
+      protected Result call(PayloadCarryingRpcController controller) throws Exception {
+        MutateRequest request = RequestConverter.buildMutateRequest(
+          getLocation().getRegionInfo().getRegionName(), append, getNonceGroup(), getNewNonce());
+        MutateResponse response = getStub().mutate(controller, request);
+        if (!response.hasResult()) return null;
+        return ProtobufUtil.toResult(response.getResult(), controller.cellScanner());
+      }
+    };
+    return rpcCallerFactory.<Result> newCaller(this.rpcTimeout).
+        callWithRetries(callable, this.operationTimeout);
   }
 
   /**
@@ -663,27 +628,16 @@ public class HTable implements Table {
    */
   @Override
   public Result increment(final Increment increment) throws IOException {
-    if (!increment.hasFamilies()) {
-      throw new IOException(
-          "Invalid arguments to increment, no columns specified");
-    }
-    NonceGenerator ng = this.connection.getNonceGenerator();
-    final long nonceGroup = ng.getNonceGroup(), nonce = ng.newNonce();
+    checkHasFamilies(increment);
     RegionServerCallable<Result> callable = new RegionServerCallable<Result>(this.connection,
-        getName(), increment.getRow()) {
+        this.rpcControllerFactory, getName(), increment.getRow()) {
       @Override
-      public Result call(int callTimeout) throws IOException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setPriority(getTableName());
-        controller.setCallTimeout(callTimeout);
-        try {
-          MutateRequest request = RequestConverter.buildMutateRequest(
-            getLocation().getRegionInfo().getRegionName(), increment, nonceGroup, nonce);
-          MutateResponse response = getStub().mutate(controller, request);
-          return ProtobufUtil.toResult(response.getResult(), controller.cellScanner());
-        } catch (ServiceException se) {
-          throw ProtobufUtil.getRemoteException(se);
-        }
+      protected Result call(PayloadCarryingRpcController controller) throws Exception {
+        MutateRequest request = RequestConverter.buildMutateRequest(
+          getLocation().getRegionInfo().getRegionName(), increment, getNonceGroup(), getNewNonce());
+        MutateResponse response = getStub().mutate(controller, request);
+        // Should this check for null like append does?
+        return ProtobufUtil.toResult(response.getResult(), controller.cellScanner());
       }
     };
     return rpcCallerFactory.<Result> newCaller(rpcTimeout).callWithRetries(callable,
@@ -722,28 +676,20 @@ public class HTable implements Table {
 
     NonceGenerator ng = this.connection.getNonceGenerator();
     final long nonceGroup = ng.getNonceGroup(), nonce = ng.newNonce();
-    RegionServerCallable<Long> callable =
-      new RegionServerCallable<Long>(connection, getName(), row) {
-        @Override
-        public Long call(int callTimeout) throws IOException {
-          PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-          controller.setPriority(getTableName());
-          controller.setCallTimeout(callTimeout);
-          try {
-            MutateRequest request = RequestConverter.buildIncrementRequest(
-              getLocation().getRegionInfo().getRegionName(), row, family,
-              qualifier, amount, durability, nonceGroup, nonce);
-            MutateResponse response = getStub().mutate(controller, request);
-            Result result =
-              ProtobufUtil.toResult(response.getResult(), controller.cellScanner());
-            return Long.valueOf(Bytes.toLong(result.getValue(family, qualifier)));
-          } catch (ServiceException se) {
-            throw ProtobufUtil.getRemoteException(se);
-          }
-        }
-      };
-    return rpcCallerFactory.<Long> newCaller(rpcTimeout).callWithRetries(callable,
-        this.operationTimeout);
+    RegionServerCallable<Long> callable = new RegionServerCallable<Long>(this.connection,
+        this.rpcControllerFactory, getName(), row) {
+      @Override
+      protected Long call(PayloadCarryingRpcController controller) throws Exception {
+        MutateRequest request = RequestConverter.buildIncrementRequest(
+          getLocation().getRegionInfo().getRegionName(), row, family,
+          qualifier, amount, durability, nonceGroup, nonce);
+        MutateResponse response = getStub().mutate(controller, request);
+        Result result = ProtobufUtil.toResult(response.getResult(), controller.cellScanner());
+        return Long.valueOf(Bytes.toLong(result.getValue(family, qualifier)));
+      }
+    };
+    return rpcCallerFactory.<Long> newCaller(rpcTimeout).
+        callWithRetries(callable, this.operationTimeout);
   }
 
   /**
@@ -754,26 +700,19 @@ public class HTable implements Table {
       final byte [] family, final byte [] qualifier, final byte [] value,
       final Put put)
   throws IOException {
-    RegionServerCallable<Boolean> callable =
-      new RegionServerCallable<Boolean>(connection, getName(), row) {
-        @Override
-        public Boolean call(int callTimeout) throws IOException {
-          PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-          controller.setPriority(tableName);
-          controller.setCallTimeout(callTimeout);
-          try {
-            MutateRequest request = RequestConverter.buildMutateRequest(
-                getLocation().getRegionInfo().getRegionName(), row, family, qualifier,
-                new BinaryComparator(value), CompareType.EQUAL, put);
-            MutateResponse response = getStub().mutate(controller, request);
-            return Boolean.valueOf(response.getProcessed());
-          } catch (ServiceException se) {
-            throw ProtobufUtil.getRemoteException(se);
-          }
-        }
-      };
-    return rpcCallerFactory.<Boolean> newCaller(rpcTimeout).callWithRetries(callable,
-        this.operationTimeout);
+    RegionServerCallable<Boolean> callable = new RegionServerCallable<Boolean>(this.connection,
+        this.rpcControllerFactory, getName(), row) {
+      @Override
+      protected Boolean call(PayloadCarryingRpcController controller) throws Exception {
+        MutateRequest request = RequestConverter.buildMutateRequest(
+          getLocation().getRegionInfo().getRegionName(), row, family, qualifier,
+          new BinaryComparator(value), CompareType.EQUAL, put);
+        MutateResponse response = getStub().mutate(controller, request);
+        return Boolean.valueOf(response.getProcessed());
+      }
+    };
+    return rpcCallerFactory.<Boolean> newCaller(rpcTimeout).
+        callWithRetries(callable, this.operationTimeout);
   }
 
   /**
@@ -784,57 +723,42 @@ public class HTable implements Table {
       final byte [] qualifier, final CompareOp compareOp, final byte [] value,
       final Put put)
   throws IOException {
-    RegionServerCallable<Boolean> callable =
-      new RegionServerCallable<Boolean>(connection, getName(), row) {
-        @Override
-        public Boolean call(int callTimeout) throws IOException {
-          PayloadCarryingRpcController controller = new PayloadCarryingRpcController();
-          controller.setPriority(tableName);
-          controller.setCallTimeout(callTimeout);
-          try {
-            CompareType compareType = CompareType.valueOf(compareOp.name());
-            MutateRequest request = RequestConverter.buildMutateRequest(
-              getLocation().getRegionInfo().getRegionName(), row, family, qualifier,
-                new BinaryComparator(value), compareType, put);
-            MutateResponse response = getStub().mutate(controller, request);
-            return Boolean.valueOf(response.getProcessed());
-          } catch (ServiceException se) {
-            throw ProtobufUtil.getRemoteException(se);
-          }
-        }
-      };
-    return rpcCallerFactory.<Boolean> newCaller(rpcTimeout).callWithRetries(callable,
-        this.operationTimeout);
+    RegionServerCallable<Boolean> callable = new RegionServerCallable<Boolean>(this.connection,
+        this.rpcControllerFactory, getName(), row) {
+      @Override
+      protected Boolean call(PayloadCarryingRpcController controller) throws Exception {
+        CompareType compareType = CompareType.valueOf(compareOp.name());
+        MutateRequest request = RequestConverter.buildMutateRequest(
+          getLocation().getRegionInfo().getRegionName(), row, family, qualifier,
+          new BinaryComparator(value), compareType, put);
+        MutateResponse response = getStub().mutate(controller, request);
+        return Boolean.valueOf(response.getProcessed());
+      }
+    };
+    return rpcCallerFactory.<Boolean> newCaller(rpcTimeout).
+        callWithRetries(callable, this.operationTimeout);
   }
 
   /**
    * {@inheritDoc}
    */
   @Override
-  public boolean checkAndDelete(final byte [] row,
-      final byte [] family, final byte [] qualifier, final byte [] value,
-      final Delete delete)
+  public boolean checkAndDelete(final byte [] row, final byte [] family, final byte [] qualifier,
+      final byte [] value, final Delete delete)
   throws IOException {
-    RegionServerCallable<Boolean> callable =
-      new RegionServerCallable<Boolean>(connection, getName(), row) {
-        @Override
-        public Boolean call(int callTimeout) throws IOException {
-          PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-          controller.setPriority(tableName);
-          controller.setCallTimeout(callTimeout);
-          try {
-            MutateRequest request = RequestConverter.buildMutateRequest(
-              getLocation().getRegionInfo().getRegionName(), row, family, qualifier,
-                new BinaryComparator(value), CompareType.EQUAL, delete);
-            MutateResponse response = getStub().mutate(controller, request);
-            return Boolean.valueOf(response.getProcessed());
-          } catch (ServiceException se) {
-            throw ProtobufUtil.getRemoteException(se);
-          }
-        }
-      };
-    return rpcCallerFactory.<Boolean> newCaller(rpcTimeout).callWithRetries(callable,
-        this.operationTimeout);
+    RegionServerCallable<Boolean> callable = new RegionServerCallable<Boolean>(this.connection,
+        this.rpcControllerFactory, getName(), row) {
+      @Override
+      protected Boolean call(PayloadCarryingRpcController controller) throws Exception {
+        MutateRequest request = RequestConverter.buildMutateRequest(
+          getLocation().getRegionInfo().getRegionName(), row, family, qualifier,
+          new BinaryComparator(value), CompareType.EQUAL, delete);
+        MutateResponse response = getStub().mutate(controller, request);
+        return Boolean.valueOf(response.getProcessed());
+      }
+    };
+    return rpcCallerFactory.<Boolean> newCaller(rpcTimeout).
+        callWithRetries(callable, this.operationTimeout);
   }
 
   /**
@@ -845,25 +769,18 @@ public class HTable implements Table {
       final byte [] qualifier, final CompareOp compareOp, final byte [] value,
       final Delete delete)
   throws IOException {
-    RegionServerCallable<Boolean> callable =
-      new RegionServerCallable<Boolean>(connection, getName(), row) {
-        @Override
-        public Boolean call(int callTimeout) throws IOException {
-          PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-          controller.setPriority(tableName);
-          controller.setCallTimeout(callTimeout);
-          try {
-            CompareType compareType = CompareType.valueOf(compareOp.name());
-            MutateRequest request = RequestConverter.buildMutateRequest(
-              getLocation().getRegionInfo().getRegionName(), row, family, qualifier,
-                new BinaryComparator(value), compareType, delete);
-            MutateResponse response = getStub().mutate(controller, request);
-            return Boolean.valueOf(response.getProcessed());
-          } catch (ServiceException se) {
-            throw ProtobufUtil.getRemoteException(se);
-          }
-        }
-      };
+    RegionServerCallable<Boolean> callable = new RegionServerCallable<Boolean>(this.connection,
+        this.rpcControllerFactory, getName(), row) {
+      @Override
+      protected Boolean call(PayloadCarryingRpcController controller) throws Exception {
+        CompareType compareType = CompareType.valueOf(compareOp.name());
+        MutateRequest request = RequestConverter.buildMutateRequest(
+          getLocation().getRegionInfo().getRegionName(), row, family, qualifier,
+          new BinaryComparator(value), compareType, delete);
+        MutateResponse response = getStub().mutate(controller, request);
+        return Boolean.valueOf(response.getProcessed());
+      }
+    };
     return rpcCallerFactory.<Boolean> newCaller(rpcTimeout).callWithRetries(callable,
         this.operationTimeout);
   }
@@ -875,40 +792,28 @@ public class HTable implements Table {
   public boolean checkAndMutate(final byte [] row, final byte [] family, final byte [] qualifier,
     final CompareOp compareOp, final byte [] value, final RowMutations rm)
     throws IOException {
-    final RetryingTimeTracker tracker = new RetryingTimeTracker();
     PayloadCarryingServerCallable<MultiResponse> callable =
       new PayloadCarryingServerCallable<MultiResponse>(connection, getName(), rm.getRow(),
         rpcControllerFactory) {
         @Override
-        public MultiResponse call(int callTimeout) throws IOException {
-          tracker.start();
-          controller.setPriority(tableName);
-          int remainingTime = tracker.getRemainingTime(callTimeout);
-          if (remainingTime == 0) {
-            throw new DoNotRetryIOException("Timeout for mutate row");
-          }
-          controller.setCallTimeout(remainingTime);
-          try {
-            CompareType compareType = CompareType.valueOf(compareOp.name());
-            MultiRequest request = RequestConverter.buildMutateRequest(
-              getLocation().getRegionInfo().getRegionName(), row, family, qualifier,
-              new BinaryComparator(value), compareType, rm);
-            ClientProtos.MultiResponse response = getStub().multi(controller, request);
-            ClientProtos.RegionActionResult res = response.getRegionActionResultList().get(0);
-            if (res.hasException()) {
-              Throwable ex = ProtobufUtil.toException(res.getException());
-              if(ex instanceof IOException) {
-                throw (IOException)ex;
-              }
-              throw new IOException("Failed to checkAndMutate row: "+
-                                    Bytes.toStringBinary(rm.getRow()), ex);
+        protected MultiResponse call(PayloadCarryingRpcController controller) throws Exception {
+          CompareType compareType = CompareType.valueOf(compareOp.name());
+          MultiRequest request = RequestConverter.buildMutateRequest(
+            getLocation().getRegionInfo().getRegionName(), row, family, qualifier,
+            new BinaryComparator(value), compareType, rm);
+          ClientProtos.MultiResponse response = getStub().multi(controller, request);
+          ClientProtos.RegionActionResult res = response.getRegionActionResultList().get(0);
+          if (res.hasException()) {
+            Throwable ex = ProtobufUtil.toException(res.getException());
+            if (ex instanceof IOException) {
+              throw (IOException)ex;
             }
-            return ResponseConverter.getResults(request, response, controller.cellScanner());
-          } catch (ServiceException se) {
-            throw ProtobufUtil.getRemoteException(se);
+            throw new IOException("Failed to checkAndMutate row: "+ Bytes.toStringBinary(rm.getRow()), ex);
           }
+          return ResponseConverter.getResults(request, response, controller.cellScanner());
         }
       };
+
     /**
      *  Currently, we use one array to store 'processed' flag which is returned by server.
      *  It is excessive to send such a large array, but that is required by the framework right now
@@ -968,7 +873,6 @@ public class HTable implements Table {
   }
 
   /**
-   * {@inheritDoc}
    * @throws IOException
    */
   void flushCommits() throws IOException {
@@ -1145,19 +1049,18 @@ public class HTable implements Table {
     for (final byte[] r : keys) {
       final RegionCoprocessorRpcChannel channel =
           new RegionCoprocessorRpcChannel(connection, tableName, r);
-      Future<R> future = pool.submit(
-          new Callable<R>() {
-            @Override
-            public R call() throws Exception {
-              T instance = ProtobufUtil.newServiceStub(service, channel);
-              R result = callable.call(instance);
-              byte[] region = channel.getLastRegion();
-              if (callback != null) {
-                callback.update(region, r, result);
-              }
-              return result;
-            }
-          });
+      Future<R> future = pool.submit(new Callable<R>() {
+        @Override
+        public R call() throws Exception {
+          T instance = ProtobufUtil.newServiceStub(service, channel);
+          R result = callable.call(instance);
+          byte[] region = channel.getLastRegion();
+          if (callback != null) {
+            callback.update(region, r, result);
+          }
+          return result;
+        }
+      });
       futures.put(r, future);
     }
     for (Map.Entry<byte[],Future<R>> e : futures.entrySet()) {
@@ -1210,9 +1113,6 @@ public class HTable implements Table {
     return tableName + ";" + connection;
   }
 
-  /**
-   * {@inheritDoc}
-   */
   @Override
   public <R extends Message> Map<byte[], R> batchCoprocessorService(
       Descriptors.MethodDescriptor methodDescriptor, Message request,
@@ -1221,14 +1121,13 @@ public class HTable implements Table {
         Bytes.BYTES_COMPARATOR));
     batchCoprocessorService(methodDescriptor, request, startKey, endKey, responsePrototype,
         new Callback<R>() {
-
-          @Override
-          public void update(byte[] region, byte[] row, R result) {
-            if (region != null) {
-              results.put(region, result);
-            }
-          }
-        });
+      @Override
+      public void update(byte[] region, byte[] row, R result) {
+        if (region != null) {
+          results.put(region, result);
+        }
+      }
+    });
     return results;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/ed87a81b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterCallable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterCallable.java
index 66d3c21..ae62255 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterCallable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterCallable.java
@@ -21,16 +21,24 @@ package org.apache.hadoop.hbase.client;
 import java.io.Closeable;
 import java.io.IOException;
 
+import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
+import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+
 /**
  * A RetryingCallable for master operations.
  * @param <V> return type
  */
+// Like RegionServerCallable
 abstract class MasterCallable<V> implements RetryingCallable<V>, Closeable {
   protected ClusterConnection connection;
   protected MasterKeepAliveConnection master;
+  private final PayloadCarryingRpcController rpcController;
 
-  public MasterCallable(final Connection connection) {
+  MasterCallable(final Connection connection,
+      final RpcControllerFactory rpcConnectionFactory) {
     this.connection = (ClusterConnection) connection;
+    this.rpcController = rpcConnectionFactory.newController();
   }
 
   @Override
@@ -59,4 +67,31 @@ abstract class MasterCallable<V> implements RetryingCallable<V>, Closeable {
   public long sleep(long pause, int tries) {
     return ConnectionUtils.getPauseTime(pause, tries);
   }
+
+  /**
+   * Override that changes Exception from {@link Exception} to {@link IOException}. It also does
+   * setup of an rpcController and calls through to the unimplemented
+   * call(PayloadCarryingRpcController) method; implement this method to add your rpc invocation.
+   */
+  @Override
+  // Same trick as in RegionServerCallable so users don't have to copy/paste so much boilerplate
+  // and so we contain references to protobuf. We can't set priority on the rpcController as
+  // we do in RegionServerCallable because we don't always have a Table when we call.
+  public V call(int callTimeout) throws IOException {
+    try {
+      this.rpcController.setCallTimeout(callTimeout);
+      return call(this.rpcController);
+    } catch (Exception e) {
+      throw ProtobufUtil.handleRemoteException(e);
+    }
+  }
+
+  /**
+   * Run RPC call.
+   * @param rpcController PayloadCarryingRpcController is a mouthful but it at a minimum is a
+   * facade on protobuf so we don't have to put protobuf everywhere; we can keep it behind this
+   * class.
+   * @throws Exception
+   */
+  protected abstract V call(PayloadCarryingRpcController rpcController) throws Exception;
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/ed87a81b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterKeepAliveConnection.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterKeepAliveConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterKeepAliveConnection.java
index e445b78..47693f4 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterKeepAliveConnection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterKeepAliveConnection.java
@@ -33,8 +33,7 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos;
  * against the master on the MasterProtos.MasterService.BlockingInterface; but not by
  * final user code. Hence it's package protected.
  */
-interface MasterKeepAliveConnection
-extends MasterProtos.MasterService.BlockingInterface {
+interface MasterKeepAliveConnection extends MasterProtos.MasterService.BlockingInterface {
   // Do this instead of implement Closeable because closeable returning IOE is PITA.
   void close();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ed87a81b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java
index e764ceb..a3162f4 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java
@@ -30,8 +30,9 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
-import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.protobuf.ResponseConverter;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
@@ -41,14 +42,14 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.protobuf.ServiceException;
 
 /**
  * Callable that handles the <code>multi</code> method call going against a single
- * regionserver; i.e. A {@link RegionServerCallable} for the multi call (It is not a
- * {@link RegionServerCallable} that goes against multiple regions.
+ * regionserver; i.e. A RegionServerCallable for the multi call (It is NOT a
+ * RegionServerCallable that goes against multiple regions).
  * @param <R>
  */
+@InterfaceAudience.Private
 class MultiServerCallable<R> extends PayloadCarryingServerCallable<MultiResponse> {
   private final MultiAction<R> multiAction;
   private final boolean cellBlock;
@@ -79,7 +80,7 @@ class MultiServerCallable<R> extends PayloadCarryingServerCallable<MultiResponse
   }
 
   @Override
-  public MultiResponse call(int callTimeout) throws IOException {
+  protected MultiResponse call(PayloadCarryingRpcController controller) throws Exception {
     int countOfActions = this.multiAction.size();
     if (countOfActions <= 0) throw new DoNotRetryIOException("No Actions");
     MultiRequest.Builder multiRequestBuilder = MultiRequest.newBuilder();
@@ -98,10 +99,8 @@ class MultiServerCallable<R> extends PayloadCarryingServerCallable<MultiResponse
       regionActionBuilder.clear();
       regionActionBuilder.setRegion(RequestConverter.buildRegionSpecifier(
           HBaseProtos.RegionSpecifier.RegionSpecifierType.REGION_NAME, regionName));
-
-
       if (this.cellBlock) {
-        // Presize.  Presume at least a KV per Action.  There are likely more.
+        // Pre-size. Presume at least a KV per Action.  There are likely more.
         if (cells == null) cells = new ArrayList<CellScannable>(countOfActions);
         // Send data in cellblocks. The call to buildNoDataMultiRequest will skip RowMutations.
         // They have already been handled above. Guess at count of cells
@@ -116,18 +115,18 @@ class MultiServerCallable<R> extends PayloadCarryingServerCallable<MultiResponse
 
     // Controller optionally carries cell data over the proxy/service boundary and also
     // optionally ferries cell response data back out again.
-    if (cells != null) controller.setCellScanner(CellUtil.createCellScanner(cells));
-    controller.setPriority(getTableName());
-    controller.setCallTimeout(callTimeout);
+    PayloadCarryingRpcController payloadCarryingRpcController = null;
+    if (cells != null) {
+      // Cast. Will fail if we have been passed wrong RpcController type.
+      payloadCarryingRpcController = (PayloadCarryingRpcController)controller;
+      payloadCarryingRpcController.setCellScanner(CellUtil.createCellScanner(cells));
+    }
     ClientProtos.MultiResponse responseProto;
     ClientProtos.MultiRequest requestProto = multiRequestBuilder.build();
-    try {
-      responseProto = getStub().multi(controller, requestProto);
-    } catch (ServiceException e) {
-      throw ProtobufUtil.getRemoteException(e);
-    }
+    responseProto = getStub().multi(controller, requestProto);
     if (responseProto == null) return null; // Occurs on cancel
-    return ResponseConverter.getResults(requestProto, responseProto, controller.cellScanner());
+    return ResponseConverter.getResults(requestProto, responseProto,
+        payloadCarryingRpcController ==  null? null: payloadCarryingRpcController.cellScanner());
   }
 
   /**
@@ -151,4 +150,4 @@ class MultiServerCallable<R> extends PayloadCarryingServerCallable<MultiResponse
   ServerName getServerName() {
     return location.getServerName();
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/ed87a81b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/PayloadCarryingServerCallable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/PayloadCarryingServerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/PayloadCarryingServerCallable.java
index d94f069..83d857b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/PayloadCarryingServerCallable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/PayloadCarryingServerCallable.java
@@ -16,33 +16,51 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 
 /**
- * This class is used to unify HTable calls with AsyncProcess Framework.
- * HTable can use AsyncProcess directly though this class.
+ * This class is used to unify HTable calls with AsyncProcess Framework. HTable can use
+ * AsyncProcess directly though this class. Also adds global timeout tracking on top of
+ * RegionServerCallable and implements Cancellable.
  */
 @InterfaceAudience.Private
-public abstract class PayloadCarryingServerCallable<T>
-    extends RegionServerCallable<T> implements Cancellable {
-  protected PayloadCarryingRpcController controller;
+abstract class PayloadCarryingServerCallable<T> extends RegionServerCallable<T>
+    implements Cancellable {
+  private final RetryingTimeTracker tracker = new RetryingTimeTracker();
+
+  PayloadCarryingServerCallable(Connection connection, TableName tableName, byte[] row,
+      RpcControllerFactory rpcControllerFactory) {
+    super(connection, rpcControllerFactory, tableName, row);
+  }
 
-  public PayloadCarryingServerCallable(Connection connection, TableName tableName, byte[] row,
-    RpcControllerFactory rpcControllerFactory) {
-    super(connection, tableName, row);
-    this.controller = rpcControllerFactory.newController();
+  /* Override so can mess with the callTimeout.
+   * (non-Javadoc)
+   * @see org.apache.hadoop.hbase.client.RegionServerCallable#rpcCall(int)
+   */
+  @Override
+  public T call(int callTimeout) throws IOException {
+    // It is expected (it seems) that tracker.start can be called multiple times (on each trip
+    // through the call when retrying). Also, we can call start and no need of a stop.
+    this.tracker.start();
+    int remainingTime = tracker.getRemainingTime(callTimeout);
+    if (remainingTime == 0) {
+      throw new DoNotRetryIOException("Timeout for mutate row");
+    }
+    return super.call(remainingTime);
   }
 
   @Override
   public void cancel() {
-    controller.startCancel();
+    getRpcController().startCancel();
   }
 
   @Override
   public boolean isCancelled() {
-    return controller.isCanceled();
+    return getRpcController().isCanceled();
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/ed87a81b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionAdminServiceCallable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionAdminServiceCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionAdminServiceCallable.java
index 54c93a0..4e347dd 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionAdminServiceCallable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionAdminServiceCallable.java
@@ -27,31 +27,30 @@ import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.RegionLocations;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
 import org.apache.hadoop.hbase.util.Bytes;
 
 /**
- * Similar to {@link RegionServerCallable} but for the AdminService interface. This service callable
+ * Similar to RegionServerCallable but for the AdminService interface. This service callable
  * assumes a Table and row and thus does region locating similar to RegionServerCallable.
+ * Works against Admin stub rather than Client stub.
  */
 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="URF_UNREAD_PUBLIC_OR_PROTECTED_FIELD",
   justification="stub used by ipc")
 @InterfaceAudience.Private
 public abstract class RegionAdminServiceCallable<T> implements RetryingCallable<T> {
-
-  protected final ClusterConnection connection;
-
-  protected final RpcControllerFactory rpcControllerFactory;
-
   protected AdminService.BlockingInterface stub;
+  protected final RpcControllerFactory rpcControllerFactory;
+  private PayloadCarryingRpcController controller = null;
 
+  protected final ClusterConnection connection;
   protected HRegionLocation location;
-
   protected final TableName tableName;
   protected final byte[] row;
   protected final int replicaId;
-
   protected final static int MIN_WAIT_DEAD_SERVER = 10000;
 
   public RegionAdminServiceCallable(ClusterConnection connection,
@@ -82,16 +81,13 @@ public abstract class RegionAdminServiceCallable<T> implements RetryingCallable<
     if (Thread.interrupted()) {
       throw new InterruptedIOException();
     }
-
     if (reload || location == null) {
       location = getLocation(!reload);
     }
-
     if (location == null) {
       // With this exception, there will be a retry.
       throw new HBaseIOException(getExceptionMessage());
     }
-
     this.setStub(connection.getAdmin(location.getServerName()));
   }
 
@@ -167,7 +163,39 @@ public abstract class RegionAdminServiceCallable<T> implements RetryingCallable<
     if (rl == null) {
       throw new RetriesExhaustedException("Can't get the locations");
     }
-
     return rl;
   }
-}
+
+  /**
+   * Override that changes Exception from {@link Exception} to {@link IOException}. It also does
+   * setup of an rpcController and calls through to the unimplemented
+   * call(PayloadCarryingRpcController) method; implement this method to add your rpc invocation.
+   */
+  @Override
+  // Same trick as in RegionServerCallable so users don't have to copy/paste so much boilerplate
+  // and so we contain references to protobuf. We can't set priority on the rpcController as
+  // we do in RegionServerCallable because we don't always have a Table when we call.
+  public T call(int callTimeout) throws IOException {
+    this.controller = rpcControllerFactory.newController();
+    this.controller.setPriority(this.tableName);
+    this.controller.setCallTimeout(callTimeout);
+    try {
+      return call(this.controller);
+    } catch (Exception e) {
+      throw ProtobufUtil.handleRemoteException(e);
+    }
+  }
+
+  PayloadCarryingRpcController getCurrentPayloadCarryingRpcController() {
+    return this.controller;
+  }
+
+  /**
+   * Run RPC call.
+   * @param rpcController PayloadCarryingRpcController is a mouthful but it at a minimum is a
+   * facade on protobuf so we don't have to put protobuf everywhere; we can keep it behind this
+   * class.
+   * @throws Exception
+   */
+  protected abstract T call(PayloadCarryingRpcController rpcController) throws Exception;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/ed87a81b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionServerCallable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionServerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionServerCallable.java
index d878bae..861b375 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionServerCallable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionServerCallable.java
@@ -1,5 +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
@@ -24,12 +23,20 @@ import java.io.IOException;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
+import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
 
 /**
- * Implementations call a RegionServer and implement {@link #call(int)}.
- * Passed to a {@link RpcRetryingCaller} so we retry on fail.
- * TODO: this class is actually tied to one region, because most of the paths make use of
+ * Implementations make an rpc call against a RegionService via a protobuf Service.
+ * Implement #rpcCall(RpcController) and then call {@link #call(int)} to
+ * trigger the rpc. The {@link #call(int)} eventually invokes your
+ * #rpcCall(RpcController) meanwhile saving you having to write a bunch of
+ * boilerplate. The {@link #call(int)} implementation is from {@link RpcRetryingCaller} so rpcs are
+ * retried on fail.
+ *
+ * <p>TODO: this class is actually tied to one region, because most of the paths make use of
  *       the regioninfo part of location when building requests. The only reason it works for
  *       multi-region requests (e.g. batch) is that they happen to not use the region parts.
  *       This could be done cleaner (e.g. having a generic parameter and 2 derived classes,
@@ -37,18 +44,27 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
  * @param <T> the class that the ServerCallable handles
  */
 @InterfaceAudience.Private
-public abstract class RegionServerCallable<T> extends AbstractRegionServerCallable<T> implements
-    RetryingCallable<T> {
-
+public abstract class RegionServerCallable<T> extends AbstractRegionServerCallable<T> {
   private ClientService.BlockingInterface stub;
+  private final PayloadCarryingRpcController rpcController;
 
   /**
    * @param connection Connection to use.
    * @param tableName Table name to which <code>row</code> belongs.
    * @param row The row we want in <code>tableName</code>.
    */
-  public RegionServerCallable(Connection connection, TableName tableName, byte [] row) {
+  public RegionServerCallable(Connection connection, RpcControllerFactory rpcControllerFactory,
+      TableName tableName, byte [] row) {
+    this(connection, rpcControllerFactory.newController(), tableName, row);
+  }
+
+  public RegionServerCallable(Connection connection, PayloadCarryingRpcController rpcController,
+      TableName tableName, byte [] row) {
     super(connection, tableName, row);
+    this.rpcController = rpcController;
+    if (this.rpcController != null) {
+      this.rpcController.setPriority(tableName);
+    }
   }
 
   void setClientByServiceName(ServerName service) throws IOException {
@@ -69,4 +85,42 @@ public abstract class RegionServerCallable<T> extends AbstractRegionServerCallab
   void setStub(final ClientService.BlockingInterface stub) {
     this.stub = stub;
   }
-}
+
+  /**
+   * Override that changes Exception from {@link Exception} to {@link IOException}. It also does
+   * setup of an rpcController and calls through to the unimplemented
+   * call(PayloadCarryingRpcController) method; implement this method to add your rpc invocation.
+   */
+  @Override
+  public T call(int callTimeout) throws IOException {
+    if (this.rpcController != null) {
+      this.rpcController.setCallTimeout(callTimeout);
+    }
+    try {
+      return call(this.rpcController);
+    } catch (Exception e) {
+      throw ProtobufUtil.handleRemoteException(e);
+    }
+  }
+
+  /**
+   * Run RPC call.
+   * @param rpcController PayloadCarryingRpcController is a mouthful but it at a minimum is a
+   * facade on protobuf so we don't have to put protobuf everywhere; we can keep it behind this
+   * class.
+   * @throws Exception
+   */
+  protected abstract T call(PayloadCarryingRpcController rpcController) throws Exception;
+
+  public PayloadCarryingRpcController getRpcController() {
+    return this.rpcController;
+  }
+
+  long getNonceGroup() {
+    return getConnection().getNonceGenerator().getNonceGroup();
+  }
+
+  long getNewNonce() {
+    return getConnection().getNonceGenerator().newNonce();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/ed87a81b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetryingTimeTracker.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetryingTimeTracker.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetryingTimeTracker.java
index 24288e6..b9438e6 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetryingTimeTracker.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetryingTimeTracker.java
@@ -22,7 +22,6 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
  * Tracks the amount of time remaining for an operation.
  */
 class RetryingTimeTracker {
-
   private long globalStartTime = -1;
 
   public void start() {
@@ -38,16 +37,19 @@ class RetryingTimeTracker {
       if (callTimeout == Integer.MAX_VALUE) {
         return Integer.MAX_VALUE;
       }
-      int remainingTime = (int) (
-        callTimeout -
-        (EnvironmentEdgeManager.currentTime() - this.globalStartTime));
+      long remaining = EnvironmentEdgeManager.currentTime() - this.globalStartTime;
+      long remainingTime = callTimeout - remaining;
       if (remainingTime < 1) {
         // If there is no time left, we're trying anyway. It's too late.
         // 0 means no timeout, and it's not the intent here. So we secure both cases by
         // resetting to the minimum.
         remainingTime = 1;
       }
-      return remainingTime;
+      if (remainingTime > Integer.MAX_VALUE) {
+        throw new RuntimeException("remainingTime=" + remainingTime +
+            " which is > Integer.MAX_VALUE");
+      }
+      return (int)remainingTime;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/ed87a81b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedScannerCallable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedScannerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedScannerCallable.java
index 0c2d345..644337d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedScannerCallable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedScannerCallable.java
@@ -176,9 +176,9 @@ public class ReversedScannerCallable extends ScannerCallable {
 
   @Override
   public ScannerCallable getScannerCallableForReplica(int id) {
-    ReversedScannerCallable r = new ReversedScannerCallable(this.cConnection, this.tableName,
+    ReversedScannerCallable r = new ReversedScannerCallable(getConnection(), this.tableName,
         this.getScan(), this.scanMetrics, this.locateStartRow, controllerFactory, id);
     r.setCaching(this.getCaching());
     return r;
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/ed87a81b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallable.java
new file mode 100644
index 0000000..68a4aa2
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallable.java
@@ -0,0 +1,65 @@
+/**
+ * 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
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.client;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+
+/**
+ * A RetryingCallable for RPC connection operations.
+ * @param <V> return type
+ */
+abstract class RpcRetryingCallable<V> implements RetryingCallable<V>, Closeable {
+  @Override
+  public void prepare(boolean reload) throws IOException {
+  }
+
+  @Override
+  public void close() throws IOException {
+  }
+
+  @Override
+  public void throwable(Throwable t, boolean retrying) {
+  }
+
+  @Override
+  public String getExceptionMessageAdditionalDetail() {
+    return "";
+  }
+
+  @Override
+  public long sleep(long pause, int tries) {
+    return ConnectionUtils.getPauseTime(pause, tries);
+  }
+
+  @Override
+  // Same trick as in RegionServerCallable so users don't have to copy/paste so much boilerplate
+  // and so we contain references to protobuf.
+  public V call(int callTimeout) throws IOException {
+    try {
+      return rpcCall(callTimeout);
+    } catch (Exception e) {
+      throw ProtobufUtil.handleRemoteException(e);
+    }
+  }
+
+  protected abstract V rpcCall(int callTimeout) throws Exception;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/ed87a81b/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 b4cd2ef..2b2e4c8 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
@@ -22,9 +22,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 import java.io.IOException;
 
-/**
- *
- */
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
 public interface RpcRetryingCaller<T> {
@@ -52,4 +49,4 @@ public interface RpcRetryingCaller<T> {
    */
   T callWithoutRetries(RetryingCallable<T> callable, int callTimeout)
   throws IOException, RuntimeException;
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/ed87a81b/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 1c723c5..f92aeae 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
@@ -36,6 +36,7 @@ public class RpcRetryingCallerFactory {
   private final int rpcTimeout;
   private final RetryingCallerInterceptor interceptor;
   private final int startLogErrorsCnt;
+  /* These below data members are UNUSED!!!*/
   private final boolean enableBackPressure;
   private ServerStatisticTracker stats;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/ed87a81b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerWithReadReplicas.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerWithReadReplicas.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerWithReadReplicas.java
index 65dbb10..2785648 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerWithReadReplicas.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerWithReadReplicas.java
@@ -29,8 +29,6 @@ import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseIOException;
@@ -46,8 +44,6 @@ import org.apache.hadoop.hbase.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 
-import com.google.protobuf.ServiceException;
-
 
 /**
  * Caller that goes to replica if the primary region does no answer within a configurable
@@ -57,8 +53,6 @@ import com.google.protobuf.ServiceException;
  */
 @InterfaceAudience.Private
 public class RpcRetryingCallerWithReadReplicas {
-  private static final Log LOG = LogFactory.getLog(RpcRetryingCallerWithReadReplicas.class);
-
   protected final ExecutorService pool;
   protected final ClusterConnection cConnection;
   protected final Configuration conf;
@@ -98,7 +92,7 @@ public class RpcRetryingCallerWithReadReplicas {
     private final PayloadCarryingRpcController controller;
 
     public ReplicaRegionServerCallable(int id, HRegionLocation location) {
-      super(RpcRetryingCallerWithReadReplicas.this.cConnection,
+      super(RpcRetryingCallerWithReadReplicas.this.cConnection, rpcControllerFactory,
           RpcRetryingCallerWithReadReplicas.this.tableName, get.getRow());
       this.id = id;
       this.location = location;
@@ -141,28 +135,20 @@ public class RpcRetryingCallerWithReadReplicas {
     }
 
     @Override
-    public Result call(int callTimeout) throws Exception {
+    protected Result call(PayloadCarryingRpcController controller) throws Exception {
       if (controller.isCanceled()) return null;
-
       if (Thread.interrupted()) {
         throw new InterruptedIOException();
       }
-
       byte[] reg = location.getRegionInfo().getRegionName();
-
       ClientProtos.GetRequest request =
           RequestConverter.buildGetRequest(reg, get);
       controller.setCallTimeout(callTimeout);
-
-      try {
-        ClientProtos.GetResponse response = getStub().get(controller, request);
-        if (response == null) {
-          return null;
-        }
-        return ProtobufUtil.toResult(response.getResult(), controller.cellScanner());
-      } catch (ServiceException se) {
-        throw ProtobufUtil.getRemoteException(se);
+      ClientProtos.GetResponse response = getStub().get(controller, request);
+      if (response == null) {
+        return null;
       }
+      return ProtobufUtil.toResult(response.getResult(), controller.cellScanner());
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/ed87a81b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java
index 72d69ec..1689d11 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java
@@ -52,9 +52,6 @@ import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.net.DNS;
 
-import com.google.protobuf.ServiceException;
-import com.google.protobuf.TextFormat;
-
 /**
  * Scanner operations such as create, next, etc.
  * Used by {@link ResultScanner}s made by {@link Table}. Passed to a retrying caller such as
@@ -74,7 +71,6 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
   protected boolean renew = false;
   private Scan scan;
   private int caching = 1;
-  protected final ClusterConnection cConnection;
   protected ScanMetrics scanMetrics;
   private boolean logScannerActivity = false;
   private int logCutOffLatency = 1000;
@@ -125,9 +121,8 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
    */
   public ScannerCallable(ClusterConnection connection, TableName tableName, Scan scan,
       ScanMetrics scanMetrics, RpcControllerFactory rpcControllerFactory, int id) {
-    super(connection, tableName, scan.getStartRow());
+    super(connection, rpcControllerFactory, tableName, scan.getStartRow());
     this.id = id;
-    this.cConnection = connection;
     this.scan = scan;
     this.scanMetrics = scanMetrics;
     Configuration conf = connection.getConfiguration();
@@ -185,25 +180,16 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
     }
   }
 
-
-  @Override
-  public Result [] call(int callTimeout) throws IOException {
+  protected Result [] call(PayloadCarryingRpcController controller) throws Exception {
     if (Thread.interrupted()) {
       throw new InterruptedIOException();
     }
-
-    if (controller == null) {
-      controller = controllerFactory.newController();
-      controller.setPriority(getTableName());
-      controller.setCallTimeout(callTimeout);
-    }
-
-    if (closed) {
-      if (scannerId != -1) {
+    if (this.closed) {
+      if (this.scannerId != -1) {
         close();
       }
     } else {
-      if (scannerId == -1L) {
+      if (this.scannerId == -1L) {
         this.scannerId = openScanner();
       } else {
         Result [] rrs = null;
@@ -212,61 +198,56 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
         setHeartbeatMessage(false);
         try {
           incRPCcallsMetrics();
-          request =
-              RequestConverter.buildScanRequest(scannerId, caching, false, nextCallSeq,
+          request = RequestConverter.buildScanRequest(scannerId, caching, false, nextCallSeq,
                 this.scanMetrics != null, renew);
           ScanResponse response = null;
-          try {
-            response = getStub().scan(controller, request);
-            // Client and RS maintain a nextCallSeq number during the scan. Every next() call
-            // from client to server will increment this number in both sides. Client passes this
-            // number along with the request and at RS side both the incoming nextCallSeq and its
-            // nextCallSeq will be matched. In case of a timeout this increment at the client side
-            // should not happen. If at the server side fetching of next batch of data was over,
-            // there will be mismatch in the nextCallSeq number. Server will throw
-            // OutOfOrderScannerNextException and then client will reopen the scanner with startrow
-            // as the last successfully retrieved row.
-            // See HBASE-5974
-            nextCallSeq++;
-            long timestamp = System.currentTimeMillis();
-            setHeartbeatMessage(response.hasHeartbeatMessage() && response.getHeartbeatMessage());
-            // Results are returned via controller
-            CellScanner cellScanner = controller.cellScanner();
-            rrs = ResponseConverter.getResults(cellScanner, response);
-            if (logScannerActivity) {
-              long now = System.currentTimeMillis();
-              if (now - timestamp > logCutOffLatency) {
-                int rows = rrs == null ? 0 : rrs.length;
-                LOG.info("Took " + (now-timestamp) + "ms to fetch "
+          response = getStub().scan(controller, request);
+          // Client and RS maintain a nextCallSeq number during the scan. Every next() call
+          // from client to server will increment this number in both sides. Client passes this
+          // number along with the request and at RS side both the incoming nextCallSeq and its
+          // nextCallSeq will be matched. In case of a timeout this increment at the client side
+          // should not happen. If at the server side fetching of next batch of data was over,
+          // there will be mismatch in the nextCallSeq number. Server will throw
+          // OutOfOrderScannerNextException and then client will reopen the scanner with startrow
+          // as the last successfully retrieved row.
+          // See HBASE-5974
+          nextCallSeq++;
+          long timestamp = System.currentTimeMillis();
+          setHeartbeatMessage(response.hasHeartbeatMessage() && response.getHeartbeatMessage());
+          // Results are returned via controller
+          CellScanner cellScanner = controller.cellScanner();
+          rrs = ResponseConverter.getResults(cellScanner, response);
+          if (logScannerActivity) {
+            long now = System.currentTimeMillis();
+            if (now - timestamp > logCutOffLatency) {
+              int rows = rrs == null ? 0 : rrs.length;
+              LOG.info("Took " + (now-timestamp) + "ms to fetch "
                   + rows + " rows from scanner=" + scannerId);
-              }
-            }
-            updateServerSideMetrics(response);
-            // moreResults is only used for the case where a filter exhausts all elements
-            if (response.hasMoreResults() && !response.getMoreResults()) {
-              scannerId = -1L;
-              closed = true;
-              // Implied that no results were returned back, either.
-              return null;
             }
-            // moreResultsInRegion explicitly defines when a RS may choose to terminate a batch due
-            // to size or quantity of results in the response.
-            if (response.hasMoreResultsInRegion()) {
-              // Set what the RS said
-              setHasMoreResultsContext(true);
-              setServerHasMoreResults(response.getMoreResultsInRegion());
-            } else {
-              // Server didn't respond whether it has more results or not.
-              setHasMoreResultsContext(false);
-            }
-          } catch (ServiceException se) {
-            throw ProtobufUtil.getRemoteException(se);
+          }
+          updateServerSideMetrics(response);
+          // moreResults is only used for the case where a filter exhausts all elements
+          if (response.hasMoreResults() && !response.getMoreResults()) {
+            this.scannerId = -1L;
+            this.closed = true;
+            // Implied that no results were returned back, either.
+            return null;
+          }
+          // moreResultsInRegion explicitly defines when a RS may choose to terminate a batch due
+          // to size or quantity of results in the response.
+          if (response.hasMoreResultsInRegion()) {
+            // Set what the RS said
+            setHasMoreResultsContext(true);
+            setServerHasMoreResults(response.getMoreResultsInRegion());
+          } else {
+            // Server didn't respond whether it has more results or not.
+            setHasMoreResultsContext(false);
           }
           updateResultsMetrics(rrs);
         } catch (IOException e) {
           if (logScannerActivity) {
-            LOG.info("Got exception making request " + TextFormat.shortDebugString(request)
-              + " to " + getLocation(), e);
+            LOG.info("Got exception making request " + ProtobufUtil.toText(request) + " to " +
+                getLocation(), e);
           }
           IOException ioe = e;
           if (e instanceof RemoteException) {
@@ -275,9 +256,9 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
           if (logScannerActivity && (ioe instanceof UnknownScannerException)) {
             try {
               HRegionLocation location =
-                getConnection().relocateRegion(getTableName(), scan.getStartRow());
-              LOG.info("Scanner=" + scannerId
-                + " expired, current region location is " + location.toString());
+                  getConnection().relocateRegion(getTableName(), scan.getStartRow());
+              LOG.info("Scanner=" + scannerId + " expired, current region location is " +
+                  location.toString());
             } catch (Throwable t) {
               LOG.info("Failed to relocate region", t);
             }
@@ -376,8 +357,8 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
           RequestConverter.buildScanRequest(this.scannerId, 0, true, this.scanMetrics != null);
       try {
         getStub().scan(controller, request);
-      } catch (ServiceException se) {
-        throw ProtobufUtil.getRemoteException(se);
+      } catch (Exception e) {
+        throw ProtobufUtil.handleRemoteException(e);
       }
     } catch (IOException e) {
       LOG.warn("Ignore, probably already closed", e);
@@ -387,10 +368,8 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
 
   protected long openScanner() throws IOException {
     incRPCcallsMetrics();
-    ScanRequest request =
-      RequestConverter.buildScanRequest(
-        getLocation().getRegionInfo().getRegionName(),
-        this.scan, 0, false);
+    ScanRequest request = RequestConverter.buildScanRequest(
+        getLocation().getRegionInfo().getRegionName(), this.scan, 0, false);
     try {
       ScanResponse response = getStub().scan(controller, request);
       long id = response.getScannerId();
@@ -399,8 +378,8 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
           + " on region " + getLocation().toString());
       }
       return id;
-    } catch (ServiceException se) {
-      throw ProtobufUtil.getRemoteException(se);
+    } catch (Exception e) {
+      throw ProtobufUtil.handleRemoteException(e);
     }
   }
 
@@ -443,11 +422,6 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
     return caching;
   }
 
-  @Override
-  public ClusterConnection getConnection() {
-    return cConnection;
-  }
-
   /**
    * Set the number of rows that will be fetched on next
    * @param caching the number of rows for caching
@@ -488,4 +462,4 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
   protected void setHasMoreResultsContext(boolean serverHasMoreResultsContext) {
     this.serverHasMoreResultsContext = serverHasMoreResultsContext;
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/ed87a81b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SecureBulkLoadClient.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SecureBulkLoadClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SecureBulkLoadClient.java
index 7b1547d..d6896e1 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SecureBulkLoadClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SecureBulkLoadClient.java
@@ -22,6 +22,9 @@ import java.io.IOException;
 import java.util.List;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
+import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
@@ -38,41 +41,35 @@ import org.apache.hadoop.hbase.security.SecureBulkLoadUtil;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.security.token.Token;
 
-import com.google.protobuf.ServiceException;
-
 /**
  * Client proxy for SecureBulkLoadProtocol
  */
 @InterfaceAudience.Private
 public class SecureBulkLoadClient {
   private Table table;
+  private final RpcControllerFactory rpcControllerFactory;
 
-  public SecureBulkLoadClient(Table table) {
+  public SecureBulkLoadClient(final Configuration conf, Table table) {
     this.table = table;
+    this.rpcControllerFactory = new RpcControllerFactory(conf);
   }
 
   public String prepareBulkLoad(final Connection conn) throws IOException {
     try {
-      RegionServerCallable<String> callable =
-          new RegionServerCallable<String>(conn, table.getName(), HConstants.EMPTY_START_ROW) {
-            @Override
-            public String call(int callTimeout) throws IOException {
-              byte[] regionName = getLocation().getRegionInfo().getRegionName();
-              RegionSpecifier region =
-                  RequestConverter
-                      .buildRegionSpecifier(RegionSpecifierType.REGION_NAME, regionName);
-              try {
-                PrepareBulkLoadRequest request =
-                    PrepareBulkLoadRequest.newBuilder()
-                        .setTableName(ProtobufUtil.toProtoTableName(table.getName()))
-                        .setRegion(region).build();
-                PrepareBulkLoadResponse response = getStub().prepareBulkLoad(null, request);
-                return response.getBulkToken();
-              } catch (ServiceException se) {
-                throw ProtobufUtil.getRemoteException(se);
-              }
-            }
-          };
+      RegionServerCallable<String> callable = new RegionServerCallable<String>(conn,
+          this.rpcControllerFactory, table.getName(), HConstants.EMPTY_START_ROW) {
+        @Override
+        protected String call(PayloadCarryingRpcController controller) throws Exception {
+          byte[] regionName = getLocation().getRegionInfo().getRegionName();
+          RegionSpecifier region =
+              RequestConverter.buildRegionSpecifier(RegionSpecifierType.REGION_NAME, regionName);
+          PrepareBulkLoadRequest request = PrepareBulkLoadRequest.newBuilder()
+              .setTableName(ProtobufUtil.toProtoTableName(table.getName()))
+              .setRegion(region).build();
+          PrepareBulkLoadResponse response = getStub().prepareBulkLoad(null, request);
+          return response.getBulkToken();
+        }
+      };
       return RpcRetryingCallerFactory.instantiate(conn.getConfiguration(), null)
           .<String> newCaller().callWithRetries(callable, Integer.MAX_VALUE);
     } catch (Throwable throwable) {
@@ -82,24 +79,19 @@ public class SecureBulkLoadClient {
 
   public void cleanupBulkLoad(final Connection conn, final String bulkToken) throws IOException {
     try {
-      RegionServerCallable<Void> callable =
-          new RegionServerCallable<Void>(conn, table.getName(), HConstants.EMPTY_START_ROW) {
-            @Override
-            public Void call(int callTimeout) throws IOException {
-              byte[] regionName = getLocation().getRegionInfo().getRegionName();
-              RegionSpecifier region = RequestConverter.buildRegionSpecifier(
-                RegionSpecifierType.REGION_NAME, regionName);
-              try {
-                CleanupBulkLoadRequest request =
-                    CleanupBulkLoadRequest.newBuilder().setRegion(region)
-                        .setBulkToken(bulkToken).build();
-                getStub().cleanupBulkLoad(null, request);
-              } catch (ServiceException se) {
-                throw ProtobufUtil.getRemoteException(se);
-              }
-              return null;
-            }
-          };
+      RegionServerCallable<Void> callable = new RegionServerCallable<Void>(conn,
+          this.rpcControllerFactory, table.getName(), HConstants.EMPTY_START_ROW) {
+        @Override
+        protected Void call(PayloadCarryingRpcController controller) throws Exception {
+          byte[] regionName = getLocation().getRegionInfo().getRegionName();
+          RegionSpecifier region = RequestConverter.buildRegionSpecifier(
+              RegionSpecifierType.REGION_NAME, regionName);
+          CleanupBulkLoadRequest request =
+              CleanupBulkLoadRequest.newBuilder().setRegion(region).setBulkToken(bulkToken).build();
+          getStub().cleanupBulkLoad(null, request);
+          return null;
+        }
+      };
       RpcRetryingCallerFactory.instantiate(conn.getConfiguration(), null)
           .<Void> newCaller().callWithRetries(callable, Integer.MAX_VALUE);
     } catch (Throwable throwable) {
@@ -130,12 +122,12 @@ public class SecureBulkLoadClient {
     try {
       BulkLoadHFileResponse response = client.bulkLoadHFile(null, request);
       return response.getLoaded();
-    } catch (ServiceException se) {
-      throw ProtobufUtil.getRemoteException(se);
+    } catch (Exception se) {
+      throw ProtobufUtil.handleRemoteException(se);
     }
   }
 
   public Path getStagingPath(String bulkToken, byte[] family) throws IOException {
     return SecureBulkLoadUtil.getStagingPath(table.getConfiguration(), bulkToken, family);
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/ed87a81b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/MasterCoprocessorRpcChannel.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/MasterCoprocessorRpcChannel.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/MasterCoprocessorRpcChannel.java
index 6fae5cb..a6384e3 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/MasterCoprocessorRpcChannel.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/MasterCoprocessorRpcChannel.java
@@ -77,5 +77,4 @@ public class MasterCoprocessorRpcChannel extends SyncCoprocessorRpcChannel {
     }
     return response;
   }
-
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/ed87a81b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/PayloadCarryingRpcController.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/PayloadCarryingRpcController.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/PayloadCarryingRpcController.java
index f4f18b3..6c290a6 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/PayloadCarryingRpcController.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/PayloadCarryingRpcController.java
@@ -17,24 +17,39 @@
  */
 package org.apache.hadoop.hbase.ipc;
 
+import java.io.IOException;
 import java.util.List;
+import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.hadoop.hbase.CellScannable;
 import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import com.google.protobuf.RpcCallback;
+import com.google.protobuf.RpcController;
 
 /**
  * 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.  This class is used ferrying data across the proxy/protobuf
- * service chasm.  Used by client and server ipc'ing.
+ * 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.
  */
 @InterfaceAudience.Private
-public class PayloadCarryingRpcController
-    extends TimeLimitedRpcController implements CellScannable {
+public class PayloadCarryingRpcController implements RpcController, CellScannable {
+  /**
+   * The time, in ms before the call should expire.
+   */
+  protected volatile Integer callTimeout;
+  protected volatile boolean cancelled = false;
+  protected final AtomicReference<RpcCallback<Object>> cancellationCb = new AtomicReference<>(null);
+  protected final AtomicReference<RpcCallback<IOException>> failureCb = new AtomicReference<>(null);
+  private IOException exception;
 
   public static final int PRIORITY_UNSET = -1;
   /**
@@ -93,15 +108,123 @@ public class PayloadCarryingRpcController
   }
 
   /**
+   * @param regionName RegionName. If hbase:meta, we'll set high priority.
+   */
+  public void setPriority(final byte [] regionName) {
+    if (isMetaRegion(regionName)) {
+      setPriority(TableName.META_TABLE_NAME);
+    }
+  }
+
+  private static boolean isMetaRegion(final byte[] regionName) {
+    return Bytes.equals(regionName, HRegionInfo.FIRST_META_REGIONINFO.getRegionName())
+        || Bytes.equals(regionName, HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes());
+  }
+
+  /**
    * @return The priority of this request
    */
   public int getPriority() {
     return priority;
   }
 
-  @Override public void reset() {
-    super.reset();
+  @Override
+  public void reset() {
     priority = 0;
     cellScanner = null;
+    exception = null;
+    cancelled = false;
+    failureCb.set(null);
+    cancellationCb.set(null);
+    callTimeout = null;
+  }
+
+  public int getCallTimeout() {
+    if (callTimeout != null) {
+      return callTimeout;
+    } else {
+      return 0;
+    }
+  }
+
+  public void setCallTimeout(int callTimeout) {
+    this.callTimeout = callTimeout;
+  }
+
+  public boolean hasCallTimeout(){
+    return callTimeout != null;
+  }
+
+  @Override
+  public String errorText() {
+    if (exception != null) {
+      return exception.getMessage();
+    } else {
+      return null;
+    }
+  }
+
+  /**
+   * For use in async rpc clients
+   * @return true if failed
+   */
+  @Override
+  public boolean failed() {
+    return this.exception != null;
+  }
+
+  @Override
+  public boolean isCanceled() {
+    return cancelled;
+  }
+
+  @Override
+  public void notifyOnCancel(RpcCallback<Object> cancellationCb) {
+    this.cancellationCb.set(cancellationCb);
+    if (this.cancelled) {
+      cancellationCb.run(null);
+    }
+  }
+
+  /**
+   * Notify a callback on error.
+   * For use in async rpc clients
+   *
+   * @param failureCb the callback to call on error
+   */
+  public void notifyOnFail(RpcCallback<IOException> failureCb) {
+    this.failureCb.set(failureCb);
+    if (this.exception != null) {
+      failureCb.run(this.exception);
+    }
+  }
+
+  @Override
+  public void setFailed(String reason) {
+    this.exception = new IOException(reason);
+    if (this.failureCb.get() != null) {
+      this.failureCb.get().run(this.exception);
+    }
+  }
+
+  /**
+   * Set failed with an exception to pass on.
+   * For use in async rpc clients
+   *
+   * @param e exception to set with
+   */
+  public void setFailed(IOException e) {
+    this.exception = e;
+    if (this.failureCb.get() != null) {
+      this.failureCb.get().run(this.exception);
+    }
+  }
+
+  @Override
+  public void startCancel() {
+    cancelled = true;
+    if (cancellationCb.get() != null) {
+      cancellationCb.get().run(null);
+    }
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/ed87a81b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionCoprocessorRpcChannel.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionCoprocessorRpcChannel.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionCoprocessorRpcChannel.java
index 55d6375..dbc9041 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionCoprocessorRpcChannel.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionCoprocessorRpcChannel.java
@@ -76,30 +76,23 @@ public class RegionCoprocessorRpcChannel extends SyncCoprocessorRpcChannel {
       Descriptors.MethodDescriptor method, Message request, Message responsePrototype)
           throws IOException {
     if (LOG.isTraceEnabled()) {
-      LOG.trace("Call: "+method.getName()+", "+request.toString());
+      LOG.trace("Call: " + method.getName() + ", " + request.toString());
     }
-
     if (row == null) {
       throw new IllegalArgumentException("Missing row property for remote region location");
     }
-
-    final RpcController rpcController = controller == null
-        ? rpcControllerFactory.newController() : controller;
-
     final ClientProtos.CoprocessorServiceCall call =
         CoprocessorRpcUtils.buildServiceCall(row, method, request);
     RegionServerCallable<CoprocessorServiceResponse> callable =
-        new RegionServerCallable<CoprocessorServiceResponse>(connection, table, row) {
+        new RegionServerCallable<CoprocessorServiceResponse>(connection,
+          controller == null? this.rpcControllerFactory.newController():
+            (PayloadCarryingRpcController)controller,
+          table, row) {
       @Override
-      public CoprocessorServiceResponse call(int callTimeout) throws Exception {
-        if (rpcController instanceof PayloadCarryingRpcController) {
-          ((PayloadCarryingRpcController) rpcController).setPriority(tableName);
-        }
-        if (rpcController instanceof TimeLimitedRpcController) {
-          ((TimeLimitedRpcController) rpcController).setCallTimeout(callTimeout);
-        }
+      protected CoprocessorServiceResponse call(PayloadCarryingRpcController controller)
+      throws Exception {
         byte[] regionName = getLocation().getRegionInfo().getRegionName();
-        return ProtobufUtil.execService(rpcController, getStub(), call, regionName);
+        return ProtobufUtil.execService(getRpcController(), getStub(), call, regionName);
       }
     };
     CoprocessorServiceResponse result = rpcCallerFactory.<CoprocessorServiceResponse> newCaller()


[03/50] [abbrv] hbase git commit: HBASE-16287 LruBlockCache size should not exceed acceptableSize too many(Yu Sun)

Posted by sy...@apache.org.
HBASE-16287 LruBlockCache size should not exceed acceptableSize too many(Yu Sun)


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

Branch: refs/heads/hbase-12439
Commit: 550b937bcf0b5a7e030194cca7d90524a0fc0f3d
Parents: 544dc1e
Author: chenheng <ch...@apache.org>
Authored: Thu Aug 4 21:13:42 2016 +0800
Committer: chenheng <ch...@apache.org>
Committed: Thu Aug 4 21:13:42 2016 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/io/hfile/LruBlockCache.java    | 36 ++++++++++++++++++--
 .../hadoop/hbase/io/hfile/TestCacheOnWrite.java |  2 ++
 .../hbase/io/hfile/TestLruBlockCache.java       |  5 +++
 3 files changed, 40 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/550b937b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java
index 29c5922..2fd9fdf 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java
@@ -110,6 +110,10 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize {
    */
   static final String LRU_ACCEPTABLE_FACTOR_CONFIG_NAME = "hbase.lru.blockcache.acceptable.factor";
 
+  /**
+   * Hard capacity limit of cache, will reject any put if size > this * acceptable
+   */
+  static final String LRU_HARD_CAPACITY_LIMIT_FACTOR_CONFIG_NAME = "hbase.lru.blockcache.hard.capacity.limit.factor";
   static final String LRU_SINGLE_PERCENTAGE_CONFIG_NAME = "hbase.lru.blockcache.single.percentage";
   static final String LRU_MULTI_PERCENTAGE_CONFIG_NAME = "hbase.lru.blockcache.multi.percentage";
   static final String LRU_MEMORY_PERCENTAGE_CONFIG_NAME = "hbase.lru.blockcache.memory.percentage";
@@ -136,6 +140,9 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize {
   static final float DEFAULT_MULTI_FACTOR = 0.50f;
   static final float DEFAULT_MEMORY_FACTOR = 0.25f;
 
+  /** default hard capacity limit */
+  static final float DEFAULT_HARD_CAPACITY_LIMIT_FACTOR = 1.2f;
+
   static final boolean DEFAULT_IN_MEMORY_FORCE_MODE = false;
 
   /** Statistics thread */
@@ -169,6 +176,9 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize {
   /** Cache access count (sequential ID) */
   private final AtomicLong count;
 
+  /** hard capacity limit */
+  private float hardCapacityLimitFactor;
+
   /** Cache statistics */
   private final CacheStats stats;
 
@@ -226,6 +236,7 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize {
         DEFAULT_SINGLE_FACTOR,
         DEFAULT_MULTI_FACTOR,
         DEFAULT_MEMORY_FACTOR,
+        DEFAULT_HARD_CAPACITY_LIMIT_FACTOR,
         false,
         DEFAULT_MAX_BLOCK_SIZE
         );
@@ -241,6 +252,7 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize {
         conf.getFloat(LRU_SINGLE_PERCENTAGE_CONFIG_NAME, DEFAULT_SINGLE_FACTOR),
         conf.getFloat(LRU_MULTI_PERCENTAGE_CONFIG_NAME, DEFAULT_MULTI_FACTOR),
         conf.getFloat(LRU_MEMORY_PERCENTAGE_CONFIG_NAME, DEFAULT_MEMORY_FACTOR),
+        conf.getFloat(LRU_HARD_CAPACITY_LIMIT_FACTOR_CONFIG_NAME, DEFAULT_HARD_CAPACITY_LIMIT_FACTOR),
         conf.getBoolean(LRU_IN_MEMORY_FORCE_MODE_CONFIG_NAME, DEFAULT_IN_MEMORY_FORCE_MODE),
         conf.getLong(LRU_MAX_BLOCK_SIZE, DEFAULT_MAX_BLOCK_SIZE)
         );
@@ -267,7 +279,8 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize {
   public LruBlockCache(long maxSize, long blockSize, boolean evictionThread,
       int mapInitialSize, float mapLoadFactor, int mapConcurrencyLevel,
       float minFactor, float acceptableFactor, float singleFactor,
-      float multiFactor, float memoryFactor, boolean forceInMemory, long maxBlockSize) {
+      float multiFactor, float memoryFactor, float hardLimitFactor,
+      boolean forceInMemory, long maxBlockSize) {
     this.maxBlockSize = maxBlockSize;
     if(singleFactor + multiFactor + memoryFactor != 1 ||
         singleFactor < 0 || multiFactor < 0 || memoryFactor < 0) {
@@ -295,6 +308,7 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize {
     this.elements = new AtomicLong(0);
     this.overhead = calculateOverhead(maxSize, blockSize, mapConcurrencyLevel);
     this.size = new AtomicLong(this.overhead);
+    this.hardCapacityLimitFactor = hardLimitFactor;
     if(evictionThread) {
       this.evictionThread = new EvictionThread(this);
       this.evictionThread.start(); // FindBugs SC_START_IN_CTOR
@@ -357,6 +371,22 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize {
       LOG.warn(msg);
       return;
     }
+    long currentSize = size.get();
+    long currentAcceptableSize = acceptableSize();
+    long hardLimitSize = (long) (hardCapacityLimitFactor * currentAcceptableSize);
+    if (currentSize >= hardLimitSize) {
+      stats.failInsert();
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("LruBlockCache current size " + StringUtils.byteDesc(currentSize)
+          + " has exceeded acceptable size " + StringUtils.byteDesc(currentAcceptableSize) + "  too many."
+          + " the hard limit size is " + StringUtils.byteDesc(hardLimitSize) + ", failed to put cacheKey:"
+          + cacheKey + " into LruBlockCache.");
+      }
+      if (!evictionInProgress) {
+        runEviction();
+      }
+      return;
+    }
     cb = new LruCachedBlock(cacheKey, buf, count.incrementAndGet(), inMemory);
     long newSize = updateSizeMetrics(cb, false);
     map.put(cacheKey, cb);
@@ -365,7 +395,7 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize {
       long size = map.size();
       assertCounterSanity(size, val);
     }
-    if (newSize > acceptableSize() && !evictionInProgress) {
+    if (newSize > currentAcceptableSize && !evictionInProgress) {
       runEviction();
     }
   }
@@ -915,7 +945,7 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize {
 
   public final static long CACHE_FIXED_OVERHEAD = ClassSize.align(
       (4 * Bytes.SIZEOF_LONG) + (9 * ClassSize.REFERENCE) +
-      (5 * Bytes.SIZEOF_FLOAT) + (2 * Bytes.SIZEOF_BOOLEAN)
+      (6 * Bytes.SIZEOF_FLOAT) + (2 * Bytes.SIZEOF_BOOLEAN)
       + ClassSize.OBJECT);
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/550b937b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java
index 32db448..8f9c4f7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java
@@ -158,6 +158,8 @@ public class TestCacheOnWrite {
     // default
     blockcaches.add(new CacheConfig(conf).getBlockCache());
 
+    //set LruBlockCache.LRU_HARD_CAPACITY_LIMIT_FACTOR_CONFIG_NAME to 2.0f due to HBASE-16287
+    TEST_UTIL.getConfiguration().setFloat(LruBlockCache.LRU_HARD_CAPACITY_LIMIT_FACTOR_CONFIG_NAME, 2.0f);
     // memory
     BlockCache lru = new LruBlockCache(128 * 1024 * 1024, 64 * 1024, TEST_UTIL.getConfiguration());
     blockcaches.add(lru);

http://git-wip-us.apache.org/repos/asf/hbase/blob/550b937b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestLruBlockCache.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestLruBlockCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestLruBlockCache.java
index 0d8a3bb..b4dfc0c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestLruBlockCache.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestLruBlockCache.java
@@ -311,6 +311,7 @@ public class TestLruBlockCache {
         0.33f, // single
         0.33f, // multi
         0.34f, // memory
+        1.2f,  // limit
         false,
         16 * 1024 * 1024);
 
@@ -432,6 +433,7 @@ public class TestLruBlockCache {
         0.2f, // single
         0.3f, // multi
         0.5f, // memory
+        1.2f, // limit
         true,
         16 * 1024 * 1024);
 
@@ -538,6 +540,7 @@ public class TestLruBlockCache {
         0.33f, // single
         0.33f, // multi
         0.34f, // memory
+        1.2f,  // limit
         false,
         16 * 1024 * 1024);
 
@@ -601,6 +604,7 @@ public class TestLruBlockCache {
         0.33f, // single
         0.33f, // multi
         0.34f, // memory
+        1.2f,  // limit
         false,
         1024);
     CachedItem [] tooLong = generateFixedBlocks(10, 1024+5, "long");
@@ -640,6 +644,7 @@ public class TestLruBlockCache {
         0.33f, // single
         0.33f, // multi
         0.34f, // memory
+        1.2f,  // limit
         false,
         16 * 1024 * 1024);
 


[07/50] [abbrv] hbase git commit: HBASE-16303 FilterList with MUST_PASS_ONE optimization (Ram)

Posted by sy...@apache.org.
HBASE-16303 FilterList with MUST_PASS_ONE optimization (Ram)


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

Branch: refs/heads/hbase-12439
Commit: 19c609fa5e2624e537a445d9204f06e56ae782c2
Parents: 5e23b3a
Author: Ramkrishna <ra...@intel.com>
Authored: Fri Aug 5 10:58:02 2016 +0530
Committer: Ramkrishna <ra...@intel.com>
Committed: Fri Aug 5 10:58:02 2016 +0530

----------------------------------------------------------------------
 .../test/java/org/apache/hadoop/hbase/filter/TestFilter.java    | 5 +++++
 .../java/org/apache/hadoop/hbase/filter/TestFilterList.java     | 3 +++
 2 files changed, 8 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/19c609fa/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java
index 1dd6616..4b8da96 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java
@@ -315,6 +315,7 @@ public class TestFilter {
     verifyScan(s, expectedRows, expectedKeys);
   }
 
+  @Test
   public void testPrefixFilterWithReverseScan() throws Exception {
     // Grab rows from group one (half of total)
     long expectedRows = this.numRows / 2;
@@ -412,6 +413,7 @@ public class TestFilter {
 
   }
 
+  @Test
   public void testPageFilterWithReverseScan() throws Exception {
     // KVs in first 6 rows
     KeyValue[] expectedKVs = {
@@ -491,6 +493,7 @@ public class TestFilter {
     verifyScan(s, expectedRows, expectedKeys);
   }
 
+  @Test
   public void testWhileMatchFilterWithFilterRowWithReverseScan()
       throws Exception {
     final int pageSize = 4;
@@ -520,6 +523,7 @@ public class TestFilter {
         pageSize, scannerCounter);
   }
 
+  @Test
   public void testWhileMatchFilterWithFilterRowKeyWithReverseScan()
       throws Exception {
     Scan s = new Scan();
@@ -1766,6 +1770,7 @@ public class TestFilter {
         kvs.length, idx);
   }
 
+  @Test
   public void testColumnPaginationFilterColumnOffset() throws Exception {
     KeyValue [] expectedKVs = {
       // testRowOne-0

http://git-wip-us.apache.org/repos/asf/hbase/blob/19c609fa/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java
index 440c9f5..1211e39 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java
@@ -262,6 +262,7 @@ public class TestFilterList {
    * we expect to get the same result as the 'prefix' only result.
    * @throws Exception
    */
+  @Test
   public void testFilterListTwoFiltersMustPassOne() throws Exception {
     byte[] r1 = Bytes.toBytes("Row1");
     byte[] r11 = Bytes.toBytes("Row11");
@@ -294,6 +295,7 @@ public class TestFilterList {
    * we expect to get the same result as the inclusive stop result.
    * @throws Exception
    */
+  @Test
   public void testFilterListWithInclusiveStopFilteMustPassOne() throws Exception {
     byte[] r1 = Bytes.toBytes("Row1");
     byte[] r11 = Bytes.toBytes("Row11");
@@ -353,6 +355,7 @@ public class TestFilterList {
    * Test filterKeyValue logic.
    * @throws Exception
    */
+  @Test
   public void testFilterKeyValue() throws Exception {
     Filter includeFilter = new FilterBase() {
       @Override


[16/50] [abbrv] hbase git commit: HBASE-16308 Contain protobuf references Gather up the pb references into a few locations only rather than have pb references distributed all about the code base.

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/ed87a81b/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 29650ef..fa18bd8 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
@@ -18,10 +18,6 @@
  */
 package org.apache.hadoop.hbase.client;
 
-import com.google.common.annotations.VisibleForTesting;
-import com.google.protobuf.ByteString;
-import com.google.protobuf.ServiceException;
-
 import java.io.Closeable;
 import java.io.IOException;
 import java.io.InterruptedIOException;
@@ -32,6 +28,7 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
@@ -69,7 +66,6 @@ import org.apache.hadoop.hbase.ZooKeeperConnectionException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.client.security.SecurityCapability;
-import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
 import org.apache.hadoop.hbase.ipc.MasterCoprocessorRpcChannel;
@@ -183,6 +179,8 @@ import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.zookeeper.KeeperException;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * HBaseAdmin is no longer a client API. It is marked InterfaceAudience.Private indicating that
  * this is an HBase-internal class as defined in
@@ -211,10 +209,6 @@ public class HBaseAdmin implements Admin {
   private volatile Configuration conf;
   private final long pause;
   private final int numRetries;
-  // Some operations can take a long time such as disable of big table.
-  // numRetries is for 'normal' stuff... Multiply by this factor when
-  // want to wait a long time.
-  private final int retryLongerMultiplier;
   private final int syncWaitTimeout;
   private boolean aborted;
   private int operationTimeout;
@@ -239,8 +233,6 @@ public class HBaseAdmin implements Admin {
         HConstants.DEFAULT_HBASE_CLIENT_PAUSE);
     this.numRetries = this.conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
         HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
-    this.retryLongerMultiplier = this.conf.getInt(
-        "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,
@@ -262,7 +254,7 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public boolean isAborted(){
+  public boolean isAborted() {
     return this.aborted;
   }
 
@@ -274,18 +266,16 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public Future<Boolean> abortProcedureAsync(
-      final long procId,
-      final boolean mayInterruptIfRunning) throws IOException {
+  public Future<Boolean> abortProcedureAsync(final long procId, final boolean mayInterruptIfRunning)
+  throws IOException {
     Boolean abortProcResponse = executeCallable(
-      new MasterCallable<AbortProcedureResponse>(getConnection()) {
+      new MasterCallable<AbortProcedureResponse>(getConnection(), getRpcControllerFactory()) {
         @Override
-        public AbortProcedureResponse call(int callTimeout) throws ServiceException {
-          PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-          controller.setCallTimeout(callTimeout);
+        protected AbortProcedureResponse call(PayloadCarryingRpcController rpcController)
+        throws Exception {
           AbortProcedureRequest abortProcRequest =
               AbortProcedureRequest.newBuilder().setProcId(procId).build();
-          return master.abortProcedure(controller, abortProcRequest);
+          return master.abortProcedure(rpcController, abortProcRequest);
         }
       }).getIsProcedureAborted();
 
@@ -324,9 +314,9 @@ public class HBaseAdmin implements Admin {
 
   @Override
   public boolean tableExists(final TableName tableName) throws IOException {
-    return executeCallable(new ConnectionCallable<Boolean>(getConnection()) {
+    return executeCallable(new RpcRetryingCallable<Boolean>() {
       @Override
-      public Boolean call(int callTimeout) throws ServiceException, IOException {
+      protected Boolean rpcCall(int callTimeout) throws Exception {
         return MetaTableAccessor.tableExists(connection, tableName);
       }
     });
@@ -350,14 +340,15 @@ public class HBaseAdmin implements Admin {
   @Override
   public HTableDescriptor[] listTables(final Pattern pattern, final boolean includeSysTables)
       throws IOException {
-    return executeCallable(new MasterCallable<HTableDescriptor[]>(getConnection()) {
+    return executeCallable(new MasterCallable<HTableDescriptor[]>(getConnection(),
+        getRpcControllerFactory()) {
       @Override
-      public HTableDescriptor[] call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
+      protected HTableDescriptor[] call(PayloadCarryingRpcController rpcController)
+      throws Exception {
         GetTableDescriptorsRequest req =
             RequestConverter.buildGetTableDescriptorsRequest(pattern, includeSysTables);
-        return ProtobufUtil.getHTableDescriptorArray(master.getTableDescriptors(controller, req));
+        return ProtobufUtil.getHTableDescriptorArray(master.getTableDescriptors(rpcController,
+            req));
       }
     });
   }
@@ -386,14 +377,13 @@ public class HBaseAdmin implements Admin {
   @Override
   public TableName[] listTableNames(final Pattern pattern, final boolean includeSysTables)
       throws IOException {
-    return executeCallable(new MasterCallable<TableName[]>(getConnection()) {
+    return executeCallable(new MasterCallable<TableName[]>(getConnection(),
+        getRpcControllerFactory()) {
       @Override
-      public TableName[] call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
+      protected TableName[] call(PayloadCarryingRpcController rpcController) throws Exception {
         GetTableNamesRequest req =
             RequestConverter.buildGetTableNamesRequest(pattern, includeSysTables);
-        return ProtobufUtil.getTableNameArray(master.getTableNames(controller, req)
+        return ProtobufUtil.getTableNameArray(master.getTableNames(rpcController, req)
             .getTableNamesList());
       }
     });
@@ -414,27 +404,25 @@ public class HBaseAdmin implements Admin {
   static HTableDescriptor getTableDescriptor(final TableName tableName, Connection connection,
       RpcRetryingCallerFactory rpcCallerFactory, final RpcControllerFactory rpcControllerFactory,
       int operationTimeout, int rpcTimeout) throws IOException {
-      if (tableName == null) return null;
-      HTableDescriptor htd = executeCallable(new MasterCallable<HTableDescriptor>(connection) {
-        @Override
-        public HTableDescriptor call(int callTimeout) throws ServiceException {
-          PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-          controller.setCallTimeout(callTimeout);
-          GetTableDescriptorsResponse htds;
-          GetTableDescriptorsRequest req =
-                  RequestConverter.buildGetTableDescriptorsRequest(tableName);
-          htds = master.getTableDescriptors(controller, req);
-
-          if (!htds.getTableSchemaList().isEmpty()) {
-            return ProtobufUtil.convertToHTableDesc(htds.getTableSchemaList().get(0));
-          }
-          return null;
+    if (tableName == null) return null;
+    HTableDescriptor htd =
+        executeCallable(new MasterCallable<HTableDescriptor>(connection, rpcControllerFactory) {
+      @Override
+      protected HTableDescriptor call(PayloadCarryingRpcController rpcController)
+      throws Exception {
+        GetTableDescriptorsRequest req =
+            RequestConverter.buildGetTableDescriptorsRequest(tableName);
+        GetTableDescriptorsResponse htds = master.getTableDescriptors(rpcController, req);
+        if (!htds.getTableSchemaList().isEmpty()) {
+          return ProtobufUtil.convertToHTableDesc(htds.getTableSchemaList().get(0));
         }
-      }, rpcCallerFactory, operationTimeout, rpcTimeout);
-      if (htd != null) {
-        return htd;
+        return null;
       }
-      throw new TableNotFoundException(tableName.getNameAsString());
+    }, rpcCallerFactory, operationTimeout, rpcTimeout);
+    if (htd != null) {
+      return htd;
+    }
+    throw new TableNotFoundException(tableName.getNameAsString());
   }
 
   private long getPauseTime(int tries) {
@@ -502,15 +490,14 @@ public class HBaseAdmin implements Admin {
     }
 
     CreateTableResponse response = executeCallable(
-      new MasterCallable<CreateTableResponse>(getConnection()) {
+      new MasterCallable<CreateTableResponse>(getConnection(), getRpcControllerFactory()) {
         @Override
-        public CreateTableResponse call(int callTimeout) throws ServiceException {
-          PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-          controller.setCallTimeout(callTimeout);
-          controller.setPriority(desc.getTableName());
+        protected CreateTableResponse call(PayloadCarryingRpcController rpcController)
+        throws Exception {
+          rpcController.setPriority(desc.getTableName());
           CreateTableRequest request = RequestConverter.buildCreateTableRequest(
             desc, splitKeys, ng.getNonceGroup(), ng.newNonce());
-          return master.createTable(controller, request);
+          return master.createTable(rpcController, request);
         }
       });
     return new CreateTableFuture(this, desc, splitKeys, response);
@@ -554,15 +541,14 @@ public class HBaseAdmin implements Admin {
   @Override
   public Future<Void> deleteTableAsync(final TableName tableName) throws IOException {
     DeleteTableResponse response = executeCallable(
-      new MasterCallable<DeleteTableResponse>(getConnection()) {
+      new MasterCallable<DeleteTableResponse>(getConnection(), getRpcControllerFactory()) {
         @Override
-        public DeleteTableResponse call(int callTimeout) throws ServiceException {
-          PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-          controller.setCallTimeout(callTimeout);
-          controller.setPriority(tableName);
+        protected DeleteTableResponse call(PayloadCarryingRpcController rpcController)
+        throws Exception {
+          rpcController.setPriority(tableName);
           DeleteTableRequest req =
               RequestConverter.buildDeleteTableRequest(tableName, ng.getNonceGroup(),ng.newNonce());
-          return master.deleteTable(controller,req);
+          return master.deleteTable(rpcController,req);
         }
       });
     return new DeleteTableFuture(this, tableName, response);
@@ -636,16 +622,16 @@ public class HBaseAdmin implements Admin {
   public Future<Void> truncateTableAsync(final TableName tableName, final boolean preserveSplits)
       throws IOException {
     TruncateTableResponse response =
-        executeCallable(new MasterCallable<TruncateTableResponse>(getConnection()) {
+        executeCallable(new MasterCallable<TruncateTableResponse>(getConnection(),
+            getRpcControllerFactory()) {
           @Override
-          public TruncateTableResponse call(int callTimeout) throws ServiceException {
-            PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-            controller.setCallTimeout(callTimeout);
-            controller.setPriority(tableName);
+          protected TruncateTableResponse call(PayloadCarryingRpcController rpcController)
+          throws Exception {
+            rpcController.setPriority(tableName);
             LOG.info("Started truncating " + tableName);
             TruncateTableRequest req = RequestConverter.buildTruncateTableRequest(
               tableName, preserveSplits, ng.getNonceGroup(), ng.newNonce());
-            return master.truncateTable(controller, req);
+            return master.truncateTable(rpcController, req);
           }
         });
     return new TruncateTableFuture(this, tableName, preserveSplits, response);
@@ -701,17 +687,15 @@ public class HBaseAdmin implements Admin {
   public Future<Void> enableTableAsync(final TableName tableName) throws IOException {
     TableName.isLegalFullyQualifiedTableName(tableName.getName());
     EnableTableResponse response = executeCallable(
-      new MasterCallable<EnableTableResponse>(getConnection()) {
+      new MasterCallable<EnableTableResponse>(getConnection(), getRpcControllerFactory()) {
         @Override
-        public EnableTableResponse call(int callTimeout) throws ServiceException {
-          PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-          controller.setCallTimeout(callTimeout);
-          controller.setPriority(tableName);
-
+        protected EnableTableResponse call(PayloadCarryingRpcController rpcController)
+        throws Exception {
+          rpcController.setPriority(tableName);
           LOG.info("Started enable of " + tableName);
           EnableTableRequest req =
               RequestConverter.buildEnableTableRequest(tableName, ng.getNonceGroup(),ng.newNonce());
-          return master.enableTable(controller,req);
+          return master.enableTable(rpcController,req);
         }
       });
     return new EnableTableFuture(this, tableName, response);
@@ -767,18 +751,16 @@ public class HBaseAdmin implements Admin {
   public Future<Void> disableTableAsync(final TableName tableName) throws IOException {
     TableName.isLegalFullyQualifiedTableName(tableName.getName());
     DisableTableResponse response = executeCallable(
-      new MasterCallable<DisableTableResponse>(getConnection()) {
+      new MasterCallable<DisableTableResponse>(getConnection(), getRpcControllerFactory()) {
         @Override
-        public DisableTableResponse call(int callTimeout) throws ServiceException {
-          PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-          controller.setCallTimeout(callTimeout);
-          controller.setPriority(tableName);
-
+        protected DisableTableResponse call(PayloadCarryingRpcController rpcController)
+        throws Exception {
+          rpcController.setPriority(tableName);
           LOG.info("Started disable of " + tableName);
           DisableTableRequest req =
               RequestConverter.buildDisableTableRequest(
                 tableName, ng.getNonceGroup(), ng.newNonce());
-          return master.disableTable(controller, req);
+          return master.disableTable(rpcController, req);
         }
       });
     return new DisableTableFuture(this, tableName, response);
@@ -827,9 +809,9 @@ public class HBaseAdmin implements Admin {
   @Override
   public boolean isTableEnabled(final TableName tableName) throws IOException {
     checkTableExists(tableName);
-    return executeCallable(new ConnectionCallable<Boolean>(getConnection()) {
+    return executeCallable(new RpcRetryingCallable<Boolean>() {
       @Override
-      public Boolean call(int callTimeout) throws ServiceException, IOException {
+      protected Boolean rpcCall(int callTimeout) throws Exception {
         TableState tableState = MetaTableAccessor.getTableState(connection, tableName);
         if (tableState == null)
           throw new TableNotFoundException(tableName);
@@ -856,16 +838,15 @@ public class HBaseAdmin implements Admin {
 
   @Override
   public Pair<Integer, Integer> getAlterStatus(final TableName tableName) throws IOException {
-    return executeCallable(new MasterCallable<Pair<Integer, Integer>>(getConnection()) {
+    return executeCallable(new MasterCallable<Pair<Integer, Integer>>(getConnection(),
+        getRpcControllerFactory()) {
       @Override
-      public Pair<Integer, Integer> call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
-        controller.setPriority(tableName);
-
+      protected Pair<Integer, Integer> call(PayloadCarryingRpcController rpcController)
+      throws Exception {
+        rpcController.setPriority(tableName);
         GetSchemaAlterStatusRequest req = RequestConverter
             .buildGetSchemaAlterStatusRequest(tableName);
-        GetSchemaAlterStatusResponse ret = master.getSchemaAlterStatus(controller, req);
+        GetSchemaAlterStatusResponse ret = master.getSchemaAlterStatus(rpcController, req);
         Pair<Integer, Integer> pair = new Pair<>(ret.getYetToUpdateRegions(),
             ret.getTotalRegions());
         return pair;
@@ -894,17 +875,16 @@ public class HBaseAdmin implements Admin {
   public Future<Void> addColumnFamily(final TableName tableName,
       final HColumnDescriptor columnFamily) throws IOException {
     AddColumnResponse response =
-        executeCallable(new MasterCallable<AddColumnResponse>(getConnection()) {
+        executeCallable(new MasterCallable<AddColumnResponse>(getConnection(),
+            getRpcControllerFactory()) {
           @Override
-          public AddColumnResponse call(int callTimeout) throws ServiceException {
-            PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-            controller.setCallTimeout(callTimeout);
-            controller.setPriority(tableName);
-
+          protected AddColumnResponse call(PayloadCarryingRpcController rpcController)
+          throws Exception {
+            rpcController.setPriority(tableName);
             AddColumnRequest req =
                 RequestConverter.buildAddColumnRequest(tableName, columnFamily, ng.getNonceGroup(),
                   ng.newNonce());
-            return master.addColumn(controller, req);
+            return master.addColumn(rpcController, req);
           }
         });
     return new AddColumnFamilyFuture(this, tableName, response);
@@ -939,17 +919,16 @@ public class HBaseAdmin implements Admin {
   public Future<Void> deleteColumnFamily(final TableName tableName, final byte[] columnFamily)
       throws IOException {
     DeleteColumnResponse response =
-        executeCallable(new MasterCallable<DeleteColumnResponse>(getConnection()) {
+        executeCallable(new MasterCallable<DeleteColumnResponse>(getConnection(),
+            getRpcControllerFactory()) {
           @Override
-          public DeleteColumnResponse call(int callTimeout) throws ServiceException {
-            PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-            controller.setCallTimeout(callTimeout);
-            controller.setPriority(tableName);
-
+          protected DeleteColumnResponse call(PayloadCarryingRpcController rpcController)
+          throws Exception {
+            rpcController.setPriority(tableName);
             DeleteColumnRequest req =
                 RequestConverter.buildDeleteColumnRequest(tableName, columnFamily,
                   ng.getNonceGroup(), ng.newNonce());
-            master.deleteColumn(controller, req);
+            master.deleteColumn(rpcController, req);
             return null;
           }
         });
@@ -985,17 +964,16 @@ public class HBaseAdmin implements Admin {
   public Future<Void> modifyColumnFamily(final TableName tableName,
       final HColumnDescriptor columnFamily) throws IOException {
     ModifyColumnResponse response =
-        executeCallable(new MasterCallable<ModifyColumnResponse>(getConnection()) {
+        executeCallable(new MasterCallable<ModifyColumnResponse>(getConnection(),
+            getRpcControllerFactory()) {
           @Override
-          public ModifyColumnResponse call(int callTimeout) throws ServiceException {
-            PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-            controller.setCallTimeout(callTimeout);
-            controller.setPriority(tableName);
-
+          protected ModifyColumnResponse call(PayloadCarryingRpcController rpcController)
+          throws Exception {
+            rpcController.setPriority(tableName);
             ModifyColumnRequest req =
                 RequestConverter.buildModifyColumnRequest(tableName, columnFamily,
                   ng.getNonceGroup(), ng.newNonce());
-            master.modifyColumn(controller, req);
+            master.modifyColumn(rpcController, req);
             return null;
           }
         });
@@ -1044,28 +1022,26 @@ public class HBaseAdmin implements Admin {
   @Override
   public boolean closeRegionWithEncodedRegionName(final String encodedRegionName,
       final String serverName) throws IOException {
-    if (null == serverName || ("").equals(serverName.trim())) {
-      throw new IllegalArgumentException(
-          "The servername cannot be null or empty.");
+   if (null == serverName || ("").equals(serverName.trim())) {
+      throw new IllegalArgumentException("The servername cannot be null or empty.");
     }
-    ServerName sn = ServerName.valueOf(serverName);
-    AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
-    // Close the region without updating zk state.
-    CloseRegionRequest request =
-      RequestConverter.buildCloseRegionRequest(sn, encodedRegionName);
-    try {
-      PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-
-      // TODO: this does not do retries, it should. Set priority and timeout in controller
-      CloseRegionResponse response = admin.closeRegion(controller, request);
-      boolean isRegionClosed = response.getClosed();
-      if (false == isRegionClosed) {
-        LOG.error("Not able to close the region " + encodedRegionName + ".");
+    final ServerName sn = ServerName.valueOf(serverName);
+    final AdminService.BlockingInterface admin = connection.getAdmin(sn);
+    final PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+    return executeCallable(new RpcRetryingCallable<Boolean>() {
+      @Override
+      protected Boolean rpcCall(int callTimeout) throws Exception {
+        controller.setCallTimeout(callTimeout);
+        CloseRegionRequest request =
+            RequestConverter.buildCloseRegionRequest(sn, encodedRegionName);
+        CloseRegionResponse response = admin.closeRegion(controller, request);
+        boolean closed = response.getClosed();
+        if (false == closed) {
+          LOG.error("Not able to close the region " + encodedRegionName + ".");
+        }
+        return closed;
       }
-      return isRegionClosed;
-    } catch (ServiceException se) {
-      throw ProtobufUtil.getRemoteException(se);
-    }
+    });
   }
 
   @Override
@@ -1104,20 +1080,20 @@ public class HBaseAdmin implements Admin {
     if (regionServerPair.getSecond() == null) {
       throw new NoServerForRegionException(Bytes.toStringBinary(regionName));
     }
-    HRegionInfo hRegionInfo = regionServerPair.getFirst();
+    final HRegionInfo hRegionInfo = regionServerPair.getFirst();
     ServerName serverName = regionServerPair.getSecond();
-
-    PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-
-    AdminService.BlockingInterface admin = this.connection.getAdmin(serverName);
-    FlushRegionRequest request =
-        RequestConverter.buildFlushRegionRequest(hRegionInfo.getRegionName());
-    try {
-      // TODO: this does not do retries, it should. Set priority and timeout in controller
-      admin.flushRegion(controller, request);
-    } catch (ServiceException se) {
-      throw ProtobufUtil.getRemoteException(se);
-    }
+    final PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+    final AdminService.BlockingInterface admin = this.connection.getAdmin(serverName);
+    executeCallable(new RpcRetryingCallable<Void>() {
+      @Override
+      protected Void rpcCall(int callTimeout) throws Exception {
+        controller.setCallTimeout(callTimeout);
+        FlushRegionRequest request =
+            RequestConverter.buildFlushRegionRequest(hRegionInfo.getRegionName());
+        admin.flushRegion(controller, request);
+        return null;
+      }
+    });
   }
 
   /**
@@ -1268,67 +1244,45 @@ public class HBaseAdmin implements Admin {
   private void compact(final ServerName sn, final HRegionInfo hri,
       final boolean major, final byte [] family)
   throws IOException {
-    PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-    AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
-    CompactRegionRequest request =
-      RequestConverter.buildCompactRegionRequest(hri.getRegionName(), major, family);
-    try {
-      // TODO: this does not do retries, it should. Set priority and timeout in controller
-      admin.compactRegion(controller, request);
-    } catch (ServiceException se) {
-      throw ProtobufUtil.getRemoteException(se);
-    }
+    final PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+    final AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
+    executeCallable(new RpcRetryingCallable<Void>() {
+      @Override
+      protected Void rpcCall(int callTimeout) throws Exception {
+        controller.setCallTimeout(callTimeout);
+        CompactRegionRequest request =
+            RequestConverter.buildCompactRegionRequest(hri.getRegionName(), major, family);
+        admin.compactRegion(controller, request);
+        return null;
+      }
+    });
   }
 
   @Override
   public void move(final byte [] encodedRegionName, final byte [] destServerName)
-      throws IOException {
-
-    executeCallable(new MasterCallable<Void>(getConnection()) {
+  throws IOException {
+    executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
       @Override
-      public Void call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
-        // Hard to know the table name, at least check if meta
-        if (isMetaRegion(encodedRegionName)) {
-          controller.setPriority(TableName.META_TABLE_NAME);
-        }
-
-        try {
-          MoveRegionRequest request =
-              RequestConverter.buildMoveRegionRequest(encodedRegionName, destServerName);
-            master.moveRegion(controller, request);
-        } catch (DeserializationException de) {
-          LOG.error("Could not parse destination server name: " + de);
-          throw new ServiceException(new DoNotRetryIOException(de));
-        }
+      protected Void call(PayloadCarryingRpcController rpcController) throws Exception {
+        rpcController.setPriority(encodedRegionName);
+        MoveRegionRequest request =
+            RequestConverter.buildMoveRegionRequest(encodedRegionName, destServerName);
+        master.moveRegion(rpcController, request);
         return null;
       }
     });
   }
 
-  private boolean isMetaRegion(final byte[] regionName) {
-    return Bytes.equals(regionName, HRegionInfo.FIRST_META_REGIONINFO.getRegionName())
-        || Bytes.equals(regionName, HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes());
-  }
-
   @Override
-  public void assign(final byte[] regionName) throws MasterNotRunningException,
+  public void assign(final byte [] regionName) throws MasterNotRunningException,
       ZooKeeperConnectionException, IOException {
-    final byte[] toBeAssigned = getRegionName(regionName);
-    executeCallable(new MasterCallable<Void>(getConnection()) {
+    executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
       @Override
-      public Void call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
-        // Hard to know the table name, at least check if meta
-        if (isMetaRegion(regionName)) {
-          controller.setPriority(TableName.META_TABLE_NAME);
-        }
-
+      protected Void call(PayloadCarryingRpcController rpcController) throws Exception {
+        rpcController.setPriority(regionName);
         AssignRegionRequest request =
-          RequestConverter.buildAssignRegionRequest(toBeAssigned);
-        master.assignRegion(controller,request);
+            RequestConverter.buildAssignRegionRequest(getRegionName(regionName));
+        master.assignRegion(rpcController, request);
         return null;
       }
     });
@@ -1338,18 +1292,13 @@ public class HBaseAdmin implements Admin {
   public void unassign(final byte [] regionName, final boolean force)
   throws MasterNotRunningException, ZooKeeperConnectionException, IOException {
     final byte[] toBeUnassigned = getRegionName(regionName);
-    executeCallable(new MasterCallable<Void>(getConnection()) {
+    executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
       @Override
-      public Void call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
-        // Hard to know the table name, at least check if meta
-        if (isMetaRegion(regionName)) {
-          controller.setPriority(TableName.META_TABLE_NAME);
-        }
+      protected Void call(PayloadCarryingRpcController rpcController) throws Exception {
+        rpcController.setPriority(regionName);
         UnassignRegionRequest request =
-          RequestConverter.buildUnassignRegionRequest(toBeUnassigned, force);
-        master.unassignRegion(controller, request);
+            RequestConverter.buildUnassignRegionRequest(toBeUnassigned, force);
+        master.unassignRegion(rpcController, request);
         return null;
       }
     });
@@ -1358,16 +1307,11 @@ public class HBaseAdmin implements Admin {
   @Override
   public void offline(final byte [] regionName)
   throws IOException {
-    executeCallable(new MasterCallable<Void>(getConnection()) {
+    executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
       @Override
-      public Void call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
-        // Hard to know the table name, at least check if meta
-        if (isMetaRegion(regionName)) {
-          controller.setPriority(TableName.META_TABLE_NAME);
-        }
-        master.offlineRegion(controller, RequestConverter.buildOfflineRegionRequest(regionName));
+      protected Void call(PayloadCarryingRpcController rpcController) throws Exception {
+        rpcController.setPriority(regionName);
+        master.offlineRegion(rpcController, RequestConverter.buildOfflineRegionRequest(regionName));
         return null;
       }
     });
@@ -1376,56 +1320,44 @@ public class HBaseAdmin implements Admin {
   @Override
   public boolean setBalancerRunning(final boolean on, final boolean synchronous)
   throws IOException {
-    return executeCallable(new MasterCallable<Boolean>(getConnection()) {
+    return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
       @Override
-      public Boolean call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
-
+      protected Boolean call(PayloadCarryingRpcController rpcController) throws Exception {
         SetBalancerRunningRequest req =
             RequestConverter.buildSetBalancerRunningRequest(on, synchronous);
-        return master.setBalancerRunning(controller, req).getPrevBalanceValue();
+        return master.setBalancerRunning(rpcController, req).getPrevBalanceValue();
       }
     });
   }
 
   @Override
   public boolean balancer() throws IOException {
-    return executeCallable(new MasterCallable<Boolean>(getConnection()) {
+    return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
       @Override
-      public Boolean call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
-
-        return master.balance(controller,
-          RequestConverter.buildBalanceRequest(false)).getBalancerRan();
+      protected Boolean call(PayloadCarryingRpcController rpcController) throws Exception {
+        return master.balance(rpcController,
+            RequestConverter.buildBalanceRequest(false)).getBalancerRan();
       }
     });
   }
 
   @Override
   public boolean balancer(final boolean force) throws IOException {
-    return executeCallable(new MasterCallable<Boolean>(getConnection()) {
+    return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
       @Override
-      public Boolean call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
-
-        return master.balance(controller,
-          RequestConverter.buildBalanceRequest(force)).getBalancerRan();
+      protected Boolean call(PayloadCarryingRpcController rpcController) throws Exception {
+        return master.balance(rpcController,
+            RequestConverter.buildBalanceRequest(force)).getBalancerRan();
       }
     });
   }
 
   @Override
   public boolean isBalancerEnabled() throws IOException {
-    return executeCallable(new MasterCallable<Boolean>(getConnection()) {
+    return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
       @Override
-      public Boolean call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
-
-        return master.isBalancerEnabled(controller,
+      protected Boolean call(PayloadCarryingRpcController rpcController) throws Exception {
+        return master.isBalancerEnabled(rpcController,
           RequestConverter.buildIsBalancerEnabledRequest()).getEnabled();
       }
     });
@@ -1433,13 +1365,10 @@ public class HBaseAdmin implements Admin {
 
   @Override
   public boolean normalize() throws IOException {
-    return executeCallable(new MasterCallable<Boolean>(getConnection()) {
+    return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
       @Override
-      public Boolean call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
-
-        return master.normalize(controller,
+      protected Boolean call(PayloadCarryingRpcController rpcController) throws Exception {
+        return master.normalize(rpcController,
           RequestConverter.buildNormalizeRequest()).getNormalizerRan();
       }
     });
@@ -1447,13 +1376,10 @@ public class HBaseAdmin implements Admin {
 
   @Override
   public boolean isNormalizerEnabled() throws IOException {
-    return executeCallable(new MasterCallable<Boolean>(getConnection()) {
+    return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
       @Override
-      public Boolean call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
-
-        return master.isNormalizerEnabled(controller,
+      protected Boolean call(PayloadCarryingRpcController rpcController) throws Exception {
+        return master.isNormalizerEnabled(rpcController,
           RequestConverter.buildIsNormalizerEnabledRequest()).getEnabled();
       }
     });
@@ -1461,28 +1387,22 @@ public class HBaseAdmin implements Admin {
 
   @Override
   public boolean setNormalizerRunning(final boolean on) throws IOException {
-    return executeCallable(new MasterCallable<Boolean>(getConnection()) {
+    return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
       @Override
-      public Boolean call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
-
+      protected Boolean call(PayloadCarryingRpcController rpcController) throws Exception {
         SetNormalizerRunningRequest req =
           RequestConverter.buildSetNormalizerRunningRequest(on);
-        return master.setNormalizerRunning(controller, req).getPrevNormalizerValue();
+        return master.setNormalizerRunning(rpcController, req).getPrevNormalizerValue();
       }
     });
   }
 
   @Override
   public boolean enableCatalogJanitor(final boolean enable) throws IOException {
-    return executeCallable(new MasterCallable<Boolean>(getConnection()) {
+    return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
       @Override
-      public Boolean call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
-
-        return master.enableCatalogJanitor(controller,
+      protected Boolean call(PayloadCarryingRpcController rpcController) throws Exception {
+        return master.enableCatalogJanitor(rpcController,
           RequestConverter.buildEnableCatalogJanitorRequest(enable)).getPrevValue();
       }
     });
@@ -1490,13 +1410,10 @@ public class HBaseAdmin implements Admin {
 
   @Override
   public int runCatalogScan() throws IOException {
-    return executeCallable(new MasterCallable<Integer>(getConnection()) {
+    return executeCallable(new MasterCallable<Integer>(getConnection(), getRpcControllerFactory()) {
       @Override
-      public Integer call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
-
-        return master.runCatalogScan(controller,
+      protected Integer call(PayloadCarryingRpcController rpcController) throws Exception {
+        return master.runCatalogScan(rpcController,
           RequestConverter.buildCatalogScanRequest()).getScanResult();
       }
     });
@@ -1504,13 +1421,10 @@ public class HBaseAdmin implements Admin {
 
   @Override
   public boolean isCatalogJanitorEnabled() throws IOException {
-    return executeCallable(new MasterCallable<Boolean>(getConnection()) {
+    return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
       @Override
-      public Boolean call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
-
-        return master.isCatalogJanitorEnabled(controller,
+      protected Boolean call(PayloadCarryingRpcController rpcController) throws Exception {
+        return master.isCatalogJanitorEnabled(rpcController,
           RequestConverter.buildIsCatalogJanitorEnabledRequest()).getValue();
       }
     });
@@ -1616,25 +1530,19 @@ public class HBaseAdmin implements Admin {
     }
 
     DispatchMergingRegionsResponse response =
-      executeCallable(new MasterCallable<DispatchMergingRegionsResponse>(getConnection()) {
+        executeCallable(new MasterCallable<DispatchMergingRegionsResponse>(getConnection(),
+            getRpcControllerFactory()) {
       @Override
-      public DispatchMergingRegionsResponse call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
-
-        try {
-          DispatchMergingRegionsRequest request = RequestConverter
-              .buildDispatchMergingRegionsRequest(
+      protected DispatchMergingRegionsResponse call(PayloadCarryingRpcController rpcController)
+          throws Exception {
+        DispatchMergingRegionsRequest request = RequestConverter
+            .buildDispatchMergingRegionsRequest(
                 encodedNameOfRegionA,
                 encodedNameOfRegionB,
                 forcible,
                 ng.getNonceGroup(),
                 ng.newNonce());
-          return master.dispatchMergingRegions(controller, request);
-        } catch (DeserializationException de) {
-          LOG.error("Could not parse destination server name: " + de);
-          throw new ServiceException(new DoNotRetryIOException(de));
-        }
+        return master.dispatchMergingRegions(rpcController, request);
       }
     });
     return new DispatchMergingRegionsFuture(this, tableName, response);
@@ -1746,21 +1654,17 @@ public class HBaseAdmin implements Admin {
       throw new IllegalArgumentException("the specified table name '" + tableName +
         "' doesn't match with the HTD one: " + htd.getTableName());
     }
-
     ModifyTableResponse response = executeCallable(
-      new MasterCallable<ModifyTableResponse>(getConnection()) {
+      new MasterCallable<ModifyTableResponse>(getConnection(), getRpcControllerFactory()) {
         @Override
-        public ModifyTableResponse call(int callTimeout) throws ServiceException {
-          PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-          controller.setCallTimeout(callTimeout);
-          controller.setPriority(tableName);
-
+        protected ModifyTableResponse call(PayloadCarryingRpcController rpcController)
+        throws Exception {
+          rpcController.setPriority(tableName);
           ModifyTableRequest request = RequestConverter.buildModifyTableRequest(
             tableName, htd, ng.getNonceGroup(), ng.newNonce());
-          return master.modifyTable(controller, request);
+          return master.modifyTable(rpcController, request);
         }
       });
-
     return new ModifyTableFuture(this, tableName, response);
   }
 
@@ -1875,9 +1779,9 @@ public class HBaseAdmin implements Admin {
    */
   private TableName checkTableExists(final TableName tableName)
       throws IOException {
-    return executeCallable(new ConnectionCallable<TableName>(getConnection()) {
+    return executeCallable(new RpcRetryingCallable<TableName>() {
       @Override
-      public TableName call(int callTimeout) throws ServiceException, IOException {
+      protected TableName rpcCall(int callTimeout) throws Exception {
         if (!MetaTableAccessor.tableExists(connection, tableName)) {
           throw new TableNotFoundException(tableName);
         }
@@ -1888,13 +1792,11 @@ public class HBaseAdmin implements Admin {
 
   @Override
   public synchronized void shutdown() throws IOException {
-    executeCallable(new MasterCallable<Void>(getConnection()) {
+    executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
       @Override
-      public Void call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
-        controller.setPriority(HConstants.HIGH_QOS);
-        master.shutdown(controller, ShutdownRequest.newBuilder().build());
+      protected Void call(PayloadCarryingRpcController rpcController) throws Exception {
+        rpcController.setPriority(HConstants.HIGH_QOS);
+        master.shutdown(rpcController, ShutdownRequest.newBuilder().build());
         return null;
       }
     });
@@ -1902,13 +1804,11 @@ public class HBaseAdmin implements Admin {
 
   @Override
   public synchronized void stopMaster() throws IOException {
-    executeCallable(new MasterCallable<Void>(getConnection()) {
+    executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
       @Override
-      public Void call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
-        controller.setPriority(HConstants.HIGH_QOS);
-        master.stopMaster(controller, StopMasterRequest.newBuilder().build());
+      protected Void call(PayloadCarryingRpcController rpcController) throws Exception {
+        rpcController.setPriority(HConstants.HIGH_QOS);
+        master.stopMaster(rpcController, StopMasterRequest.newBuilder().build());
         return null;
       }
     });
@@ -1919,43 +1819,41 @@ public class HBaseAdmin implements Admin {
   throws IOException {
     String hostname = Addressing.parseHostname(hostnamePort);
     int port = Addressing.parsePort(hostnamePort);
-    AdminService.BlockingInterface admin =
+    final AdminService.BlockingInterface admin =
       this.connection.getAdmin(ServerName.valueOf(hostname, port, 0));
-    StopServerRequest request = RequestConverter.buildStopServerRequest(
-      "Called by admin client " + this.connection.toString());
-    PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-
-    controller.setPriority(HConstants.HIGH_QOS);
-    try {
-      // TODO: this does not do retries, it should. Set priority and timeout in controller
-      admin.stopServer(controller, request);
-    } catch (ServiceException se) {
-      throw ProtobufUtil.getRemoteException(se);
-    }
+    executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
+      @Override
+      protected Void call(PayloadCarryingRpcController rpcController) throws Exception {
+        rpcController.setPriority(HConstants.HIGH_QOS);
+        StopServerRequest request = RequestConverter.buildStopServerRequest(
+            "Called by admin client " + this.connection.toString());
+        admin.stopServer(rpcController, request);
+        return null;
+      }
+    });
   }
 
   @Override
   public boolean isMasterInMaintenanceMode() throws IOException {
-    return executeCallable(new MasterCallable<IsInMaintenanceModeResponse>(getConnection()) {
+    return executeCallable(new MasterCallable<IsInMaintenanceModeResponse>(getConnection(),
+        this.rpcControllerFactory) {
       @Override
-      public IsInMaintenanceModeResponse call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
-        return master.isMasterInMaintenanceMode(
-          controller, IsInMaintenanceModeRequest.newBuilder().build());
+      protected IsInMaintenanceModeResponse call(PayloadCarryingRpcController rpcController)
+      throws Exception {
+        return master.isMasterInMaintenanceMode(rpcController,
+            IsInMaintenanceModeRequest.newBuilder().build());
       }
     }).getInMaintenanceMode();
   }
 
   @Override
   public ClusterStatus getClusterStatus() throws IOException {
-    return executeCallable(new MasterCallable<ClusterStatus>(getConnection()) {
+    return executeCallable(new MasterCallable<ClusterStatus>(getConnection(),
+        this.rpcControllerFactory) {
       @Override
-      public ClusterStatus call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
+      protected ClusterStatus call(PayloadCarryingRpcController rpcController) throws Exception {
         GetClusterStatusRequest req = RequestConverter.buildGetClusterStatusRequest();
-        return ProtobufUtil.convert(master.getClusterStatus(controller, req).getClusterStatus());
+        return ProtobufUtil.convert(master.getClusterStatus(rpcController, req).getClusterStatus());
       }
     });
   }
@@ -1996,19 +1894,16 @@ public class HBaseAdmin implements Admin {
   public Future<Void> createNamespaceAsync(final NamespaceDescriptor descriptor)
       throws IOException {
     CreateNamespaceResponse response =
-        executeCallable(new MasterCallable<CreateNamespaceResponse>(getConnection()) {
-          @Override
-          public CreateNamespaceResponse call(int callTimeout) throws Exception {
-            PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-            controller.setCallTimeout(callTimeout);
-            // TODO: set priority based on NS?
-            return master.createNamespace(controller,
-              CreateNamespaceRequest.newBuilder()
-              .setNamespaceDescriptor(ProtobufUtil
-                .toProtoNamespaceDescriptor(descriptor)).build()
-                );
-          }
-        });
+        executeCallable(new MasterCallable<CreateNamespaceResponse>(getConnection(),
+            getRpcControllerFactory()) {
+      @Override
+      protected CreateNamespaceResponse call(PayloadCarryingRpcController rpcController)
+      throws Exception {
+        return master.createNamespace(rpcController,
+          CreateNamespaceRequest.newBuilder().setNamespaceDescriptor(ProtobufUtil.
+              toProtoNamespaceDescriptor(descriptor)).build());
+      }
+    });
     return new NamespaceFuture(this, descriptor.getName(), response.getProcId()) {
       @Override
       public String getOperationType() {
@@ -2027,16 +1922,16 @@ public class HBaseAdmin implements Admin {
   public Future<Void> modifyNamespaceAsync(final NamespaceDescriptor descriptor)
       throws IOException {
     ModifyNamespaceResponse response =
-        executeCallable(new MasterCallable<ModifyNamespaceResponse>(getConnection()) {
-          @Override
-          public ModifyNamespaceResponse call(int callTimeout) throws Exception {
-            PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-            controller.setCallTimeout(callTimeout);
-            // TODO: set priority based on NS?
-            return master.modifyNamespace(controller, ModifyNamespaceRequest.newBuilder().
-              setNamespaceDescriptor(ProtobufUtil.toProtoNamespaceDescriptor(descriptor)).build());
-          }
-        });
+        executeCallable(new MasterCallable<ModifyNamespaceResponse>(getConnection(),
+            getRpcControllerFactory()) {
+      @Override
+      protected ModifyNamespaceResponse call(PayloadCarryingRpcController rpcController)
+      throws Exception {
+        // TODO: set priority based on NS?
+        return master.modifyNamespace(rpcController, ModifyNamespaceRequest.newBuilder().
+          setNamespaceDescriptor(ProtobufUtil.toProtoNamespaceDescriptor(descriptor)).build());
+       }
+    });
     return new NamespaceFuture(this, descriptor.getName(), response.getProcId()) {
       @Override
       public String getOperationType() {
@@ -2055,16 +1950,16 @@ public class HBaseAdmin implements Admin {
   public Future<Void> deleteNamespaceAsync(final String name)
       throws IOException {
     DeleteNamespaceResponse response =
-        executeCallable(new MasterCallable<DeleteNamespaceResponse>(getConnection()) {
-          @Override
-          public DeleteNamespaceResponse call(int callTimeout) throws Exception {
-            PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-            controller.setCallTimeout(callTimeout);
-            // TODO: set priority based on NS?
-            return master.deleteNamespace(controller, DeleteNamespaceRequest.newBuilder().
-              setNamespaceName(name).build());
-          }
-        });
+        executeCallable(new MasterCallable<DeleteNamespaceResponse>(getConnection(),
+            getRpcControllerFactory()) {
+      @Override
+      protected DeleteNamespaceResponse call(PayloadCarryingRpcController rpcController)
+      throws Exception {
+        // TODO: set priority based on NS?
+        return master.deleteNamespace(rpcController, DeleteNamespaceRequest.newBuilder().
+          setNamespaceName(name).build());
+        }
+      });
     return new NamespaceFuture(this, name, response.getProcId()) {
       @Override
       public String getOperationType() {
@@ -2075,100 +1970,94 @@ public class HBaseAdmin implements Admin {
 
   @Override
   public NamespaceDescriptor getNamespaceDescriptor(final String name) throws IOException {
-    return
-        executeCallable(new MasterCallable<NamespaceDescriptor>(getConnection()) {
-          @Override
-          public NamespaceDescriptor call(int callTimeout) throws Exception {
-            PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-            controller.setCallTimeout(callTimeout);
-            return ProtobufUtil.toNamespaceDescriptor(
-              master.getNamespaceDescriptor(controller, GetNamespaceDescriptorRequest.newBuilder().
+    return executeCallable(new MasterCallable<NamespaceDescriptor>(getConnection(),
+        getRpcControllerFactory()) {
+      @Override
+      protected NamespaceDescriptor call(PayloadCarryingRpcController rpcController)
+          throws Exception {
+        return ProtobufUtil.toNamespaceDescriptor(
+            master.getNamespaceDescriptor(rpcController, GetNamespaceDescriptorRequest.newBuilder().
                 setNamespaceName(name).build()).getNamespaceDescriptor());
-          }
-        });
+      }
+    });
   }
 
   @Override
   public NamespaceDescriptor[] listNamespaceDescriptors() throws IOException {
-    return
-        executeCallable(new MasterCallable<NamespaceDescriptor[]>(getConnection()) {
-          @Override
-          public NamespaceDescriptor[] call(int callTimeout) throws Exception {
-            PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-            controller.setCallTimeout(callTimeout);
-            List<HBaseProtos.NamespaceDescriptor> list =
-                master.listNamespaceDescriptors(controller,
-                  ListNamespaceDescriptorsRequest.newBuilder().build())
-                .getNamespaceDescriptorList();
-            NamespaceDescriptor[] res = new NamespaceDescriptor[list.size()];
-            for(int i = 0; i < list.size(); i++) {
-              res[i] = ProtobufUtil.toNamespaceDescriptor(list.get(i));
-            }
-            return res;
-          }
-        });
+    return executeCallable(new MasterCallable<NamespaceDescriptor[]>(getConnection(),
+        getRpcControllerFactory()) {
+      @Override
+      protected NamespaceDescriptor[] call(PayloadCarryingRpcController rpcController)
+          throws Exception {
+        List<HBaseProtos.NamespaceDescriptor> list =
+            master.listNamespaceDescriptors(rpcController,
+              ListNamespaceDescriptorsRequest.newBuilder().build()).getNamespaceDescriptorList();
+        NamespaceDescriptor[] res = new NamespaceDescriptor[list.size()];
+        for(int i = 0; i < list.size(); i++) {
+          res[i] = ProtobufUtil.toNamespaceDescriptor(list.get(i));
+        }
+        return res;
+      }
+    });
   }
 
   @Override
   public ProcedureInfo[] listProcedures() throws IOException {
-    return
-        executeCallable(new MasterCallable<ProcedureInfo[]>(getConnection()) {
-          @Override
-          public ProcedureInfo[] call(int callTimeout) throws Exception {
-            PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-            controller.setCallTimeout(callTimeout);
-            List<ProcedureProtos.Procedure> procList = master.listProcedures(
-              controller, ListProceduresRequest.newBuilder().build()).getProcedureList();
-            ProcedureInfo[] procInfoList = new ProcedureInfo[procList.size()];
-            for (int i = 0; i < procList.size(); i++) {
-              procInfoList[i] = ProcedureUtil.convert(procList.get(i));
-            }
-            return procInfoList;
-          }
-        });
+    return executeCallable(new MasterCallable<ProcedureInfo[]>(getConnection(),
+        getRpcControllerFactory()) {
+      @Override
+      protected ProcedureInfo[] call(PayloadCarryingRpcController rpcController)
+          throws Exception {
+        List<ProcedureProtos.Procedure> procList = master.listProcedures(
+            rpcController, ListProceduresRequest.newBuilder().build()).getProcedureList();
+        ProcedureInfo[] procInfoList = new ProcedureInfo[procList.size()];
+        for (int i = 0; i < procList.size(); i++) {
+          procInfoList[i] = ProcedureUtil.convert(procList.get(i));
+        }
+        return procInfoList;
+      }
+    });
   }
 
   @Override
   public HTableDescriptor[] listTableDescriptorsByNamespace(final String name) throws IOException {
-    return
-        executeCallable(new MasterCallable<HTableDescriptor[]>(getConnection()) {
-          @Override
-          public HTableDescriptor[] call(int callTimeout) throws Exception {
-            PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-            controller.setCallTimeout(callTimeout);
-            List<TableSchema> list =
-                master.listTableDescriptorsByNamespace(controller,
-                  ListTableDescriptorsByNamespaceRequest.newBuilder().setNamespaceName(name)
-                  .build()).getTableSchemaList();
-            HTableDescriptor[] res = new HTableDescriptor[list.size()];
-            for(int i=0; i < list.size(); i++) {
-
-              res[i] = ProtobufUtil.convertToHTableDesc(list.get(i));
-            }
-            return res;
-          }
-        });
+    return executeCallable(new MasterCallable<HTableDescriptor[]>(getConnection(),
+        getRpcControllerFactory()) {
+      @Override
+      protected HTableDescriptor[] call(PayloadCarryingRpcController rpcController)
+          throws Exception {
+        List<TableSchema> list =
+            master.listTableDescriptorsByNamespace(rpcController,
+                ListTableDescriptorsByNamespaceRequest.newBuilder().setNamespaceName(name)
+                .build()).getTableSchemaList();
+        HTableDescriptor[] res = new HTableDescriptor[list.size()];
+        for(int i=0; i < list.size(); i++) {
+
+          res[i] = ProtobufUtil.convertToHTableDesc(list.get(i));
+        }
+        return res;
+      }
+    });
   }
 
   @Override
   public TableName[] listTableNamesByNamespace(final String name) throws IOException {
-    return
-        executeCallable(new MasterCallable<TableName[]>(getConnection()) {
-          @Override
-          public TableName[] call(int callTimeout) throws Exception {
-            PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-            controller.setCallTimeout(callTimeout);
-            List<HBaseProtos.TableName> tableNames =
-              master.listTableNamesByNamespace(controller, ListTableNamesByNamespaceRequest.
+    return executeCallable(new MasterCallable<TableName[]>(getConnection(),
+        getRpcControllerFactory()) {
+      @Override
+      protected TableName[] call(PayloadCarryingRpcController rpcController)
+          throws Exception {
+        List<HBaseProtos.TableName> tableNames =
+            master.listTableNamesByNamespace(rpcController, ListTableNamesByNamespaceRequest.
                 newBuilder().setNamespaceName(name).build())
-                .getTableNameList();
-            TableName[] result = new TableName[tableNames.size()];
-            for (int i = 0; i < tableNames.size(); i++) {
-              result[i] = ProtobufUtil.toTableName(tableNames.get(i));
-            }
-            return result;
-          }
-        });
+            .getTableNameList();
+        TableName[] result = new TableName[tableNames.size()];
+        for (int i = 0; i < tableNames.size(); i++) {
+          result[i] = ProtobufUtil.toTableName(tableNames.get(i));
+        }
+        return result;
+      }
+    });
   }
 
   /**
@@ -2176,10 +2065,26 @@ public class HBaseAdmin implements Admin {
    * @param conf system configuration
    * @throws MasterNotRunningException if the master is not running
    * @throws ZooKeeperConnectionException if unable to connect to zookeeper
+   * @deprecated since hbase-2.0.0 because throws a ServiceException. We don't want to have
+   * protobuf as part of our public API. Use {@link #available(Configuration)}
    */
   // Used by tests and by the Merge tool. Merge tool uses it to figure if HBase is up or not.
+  // MOB uses it too.
+  // NOTE: hbase-2.0.0 removes ServiceException from the throw.
+  @Deprecated
   public static void checkHBaseAvailable(Configuration conf)
-  throws MasterNotRunningException, ZooKeeperConnectionException, ServiceException, IOException {
+  throws MasterNotRunningException, ZooKeeperConnectionException, IOException,
+  com.google.protobuf.ServiceException {
+    available(conf);
+  }
+
+  /**
+   * Is HBase available? Throw an exception if not.
+   * @param conf system configuration
+   * @throws ZooKeeperConnectionException if unable to connect to zookeeper]
+   */
+  public static void available(final Configuration conf)
+  throws ZooKeeperConnectionException, InterruptedIOException {
     Configuration copyOfConf = HBaseConfiguration.create(conf);
     // We set it to make it fail as soon as possible if HBase is not available
     copyOfConf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1);
@@ -2191,7 +2096,6 @@ public class HBaseAdmin implements Admin {
              (ClusterConnection) ConnectionFactory.createConnection(copyOfConf);
          ZooKeeperKeepAliveConnection zkw = ((ConnectionImplementation) connection).
              getKeepAliveZooKeeperWatcher();) {
-
       // This is NASTY. FIX!!!! Dependent on internal implementation! TODO
       zkw.getRecoverableZooKeeper().getZooKeeper().exists(zkw.baseZNode, false);
       connection.isMasterRunning();
@@ -2231,14 +2135,15 @@ public class HBaseAdmin implements Admin {
   @Override
   public HTableDescriptor[] getTableDescriptorsByTableName(final List<TableName> tableNames)
   throws IOException {
-    return executeCallable(new MasterCallable<HTableDescriptor[]>(getConnection()) {
+    return executeCallable(new MasterCallable<HTableDescriptor[]>(getConnection(),
+        getRpcControllerFactory()) {
       @Override
-      public HTableDescriptor[] call(int callTimeout) throws Exception {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
+      protected HTableDescriptor[] call(PayloadCarryingRpcController rpcController)
+      throws Exception {
         GetTableDescriptorsRequest req =
             RequestConverter.buildGetTableDescriptorsRequest(tableNames);
-          return ProtobufUtil.getHTableDescriptorArray(master.getTableDescriptors(controller, req));
+          return ProtobufUtil.
+              getHTableDescriptorArray(master.getTableDescriptors(rpcController, req));
       }
     });
   }
@@ -2276,16 +2181,16 @@ public class HBaseAdmin implements Admin {
 
   private RollWALWriterResponse rollWALWriterImpl(final ServerName sn) throws IOException,
       FailedLogCloseException {
-    AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
-    RollWALWriterRequest request = RequestConverter.buildRollWALWriterRequest();
-    PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-
-    try {
-      // TODO: this does not do retries, it should. Set priority and timeout in controller
-      return admin.rollWALWriter(controller, request);
-    } catch (ServiceException se) {
-      throw ProtobufUtil.getRemoteException(se);
-    }
+    final AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
+    Callable<RollWALWriterResponse> callable = new Callable<RollWALWriterResponse>() {
+      @Override
+      public RollWALWriterResponse call() throws Exception {
+        RollWALWriterRequest request = RequestConverter.buildRollWALWriterRequest();
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        return admin.rollWALWriter(controller, request);
+      }
+    };
+    return ProtobufUtil.call(callable);
   }
 
   /**
@@ -2321,8 +2226,7 @@ public class HBaseAdmin implements Admin {
     }
     byte[][] regionsToFlush = new byte[regionCount][];
     for (int i = 0; i < regionCount; i++) {
-      ByteString region = response.getRegionToFlush(i);
-      regionsToFlush[i] = region.toByteArray();
+      regionsToFlush[i] = ProtobufUtil.toBytes(response.getRegionToFlush(i));
     }
     return regionsToFlush;
   }
@@ -2352,28 +2256,31 @@ public class HBaseAdmin implements Admin {
   @Override
   public CompactionState getCompactionStateForRegion(final byte[] regionName)
   throws IOException {
-    try {
-      Pair<HRegionInfo, ServerName> regionServerPair = getRegion(regionName);
-      if (regionServerPair == null) {
-        throw new IllegalArgumentException("Invalid region: " + Bytes.toStringBinary(regionName));
-      }
-      if (regionServerPair.getSecond() == null) {
-        throw new NoServerForRegionException(Bytes.toStringBinary(regionName));
-      }
-      ServerName sn = regionServerPair.getSecond();
-      AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
-      GetRegionInfoRequest request = RequestConverter.buildGetRegionInfoRequest(
-        regionServerPair.getFirst().getRegionName(), true);
-      PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-      // TODO: this does not do retries, it should. Set priority and timeout in controller
-      GetRegionInfoResponse response = admin.getRegionInfo(controller, request);
-      if (response.getCompactionState() != null) {
-        return ProtobufUtil.createCompactionState(response.getCompactionState());
-      }
-      return null;
-    } catch (ServiceException se) {
-      throw ProtobufUtil.getRemoteException(se);
+    final Pair<HRegionInfo, ServerName> regionServerPair = getRegion(regionName);
+    if (regionServerPair == null) {
+      throw new IllegalArgumentException("Invalid region: " + Bytes.toStringBinary(regionName));
+    }
+    if (regionServerPair.getSecond() == null) {
+      throw new NoServerForRegionException(Bytes.toStringBinary(regionName));
     }
+    ServerName sn = regionServerPair.getSecond();
+    final AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
+    Callable<CompactionState> callable = new Callable<CompactionState>() {
+      @Override
+      public CompactionState call() throws Exception {
+        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+        GetRegionInfoRequest request = RequestConverter.buildGetRegionInfoRequest(
+            regionServerPair.getFirst().getRegionName(), true);
+
+        // TODO: this does not do retries, it should. Set priority and timeout in controller
+        GetRegionInfoResponse response = admin.getRegionInfo(controller, request);
+        if (response.getCompactionState() != null) {
+          return ProtobufUtil.createCompactionState(response.getCompactionState());
+        }
+        return null;
+      }
+    };
+    return ProtobufUtil.call(callable);
   }
 
   @Override
@@ -2425,12 +2332,12 @@ public class HBaseAdmin implements Admin {
         throw (InterruptedIOException)new InterruptedIOException("Interrupted").initCause(e);
       }
       LOG.debug("Getting current status of snapshot from master...");
-      done = executeCallable(new MasterCallable<IsSnapshotDoneResponse>(getConnection()) {
+      done = executeCallable(new MasterCallable<IsSnapshotDoneResponse>(getConnection(),
+          getRpcControllerFactory()) {
         @Override
-        public IsSnapshotDoneResponse call(int callTimeout) throws ServiceException {
-          PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-          controller.setCallTimeout(callTimeout);
-          return master.isSnapshotDone(controller, request);
+        protected IsSnapshotDoneResponse call(PayloadCarryingRpcController rpcController)
+        throws Exception {
+          return master.isSnapshotDone(rpcController, request);
         }
       });
     }
@@ -2476,12 +2383,12 @@ public class HBaseAdmin implements Admin {
     final SnapshotRequest request = SnapshotRequest.newBuilder().setSnapshot(snapshot)
         .build();
     // run the snapshot on the master
-    return executeCallable(new MasterCallable<SnapshotResponse>(getConnection()) {
+    return executeCallable(new MasterCallable<SnapshotResponse>(getConnection(),
+        getRpcControllerFactory()) {
       @Override
-      public SnapshotResponse call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
-        return master.snapshot(controller, request);
+      protected SnapshotResponse call(PayloadCarryingRpcController rpcController)
+      throws Exception {
+        return master.snapshot(rpcController, request);
       }
     });
   }
@@ -2490,12 +2397,12 @@ public class HBaseAdmin implements Admin {
   public boolean isSnapshotFinished(final SnapshotDescription snapshotDesc)
       throws IOException, HBaseSnapshotException, UnknownSnapshotException {
     final HBaseProtos.SnapshotDescription snapshot = createHBaseProtosSnapshotDesc(snapshotDesc);
-    return executeCallable(new MasterCallable<IsSnapshotDoneResponse>(getConnection()) {
+    return executeCallable(new MasterCallable<IsSnapshotDoneResponse>(getConnection(),
+        getRpcControllerFactory()) {
       @Override
-      public IsSnapshotDoneResponse call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
-        return master.isSnapshotDone(controller,
+      protected IsSnapshotDoneResponse call(PayloadCarryingRpcController rpcController)
+      throws Exception {
+        return master.isSnapshotDone(rpcController,
           IsSnapshotDoneRequest.newBuilder().setSnapshot(snapshot).build());
       }
     }).getDone();
@@ -2674,12 +2581,11 @@ public class HBaseAdmin implements Admin {
         .setProcedure(builder.build()).build();
     // run the procedure on the master
     ExecProcedureResponse response = executeCallable(new MasterCallable<ExecProcedureResponse>(
-        getConnection()) {
+        getConnection(), getRpcControllerFactory()) {
       @Override
-      public ExecProcedureResponse call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
-        return master.execProcedureWithRet(controller, request);
+      protected ExecProcedureResponse call(PayloadCarryingRpcController rpcController)
+      throws Exception {
+        return master.execProcedureWithRet(rpcController, request);
       }
     });
 
@@ -2701,12 +2607,11 @@ public class HBaseAdmin implements Admin {
         .setProcedure(builder.build()).build();
     // run the procedure on the master
     ExecProcedureResponse response = executeCallable(new MasterCallable<ExecProcedureResponse>(
-        getConnection()) {
+        getConnection(), getRpcControllerFactory()) {
       @Override
-      public ExecProcedureResponse call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
-        return master.execProcedure(controller, request);
+      protected ExecProcedureResponse call(PayloadCarryingRpcController rpcController)
+      throws Exception {
+        return master.execProcedure(rpcController, request);
       }
     });
 
@@ -2750,12 +2655,11 @@ public class HBaseAdmin implements Admin {
     }
     final ProcedureDescription desc = builder.build();
     return executeCallable(
-        new MasterCallable<IsProcedureDoneResponse>(getConnection()) {
+        new MasterCallable<IsProcedureDoneResponse>(getConnection(), getRpcControllerFactory()) {
           @Override
-          public IsProcedureDoneResponse call(int callTimeout) throws ServiceException {
-            PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-            controller.setCallTimeout(callTimeout);
-            return master.isProcedureDone(controller, IsProcedureDoneRequest
+          protected IsProcedureDoneResponse call(PayloadCarryingRpcController rpcController)
+          throws Exception {
+            return master.isProcedureDone(rpcController, IsProcedureDoneRequest
                 .newBuilder().setProcedure(desc).build());
           }
         }).getDone();
@@ -2781,17 +2685,16 @@ public class HBaseAdmin implements Admin {
     ClientSnapshotDescriptionUtils.assertSnapshotRequestIsValid(snapshot);
 
     RestoreSnapshotResponse response = executeCallable(
-        new MasterCallable<RestoreSnapshotResponse>(getConnection()) {
+        new MasterCallable<RestoreSnapshotResponse>(getConnection(), getRpcControllerFactory()) {
       @Override
-      public RestoreSnapshotResponse call(int callTimeout) throws ServiceException {
+      protected RestoreSnapshotResponse call(PayloadCarryingRpcController rpcController)
+      throws Exception {
         final RestoreSnapshotRequest request = RestoreSnapshotRequest.newBuilder()
             .setSnapshot(snapshot)
             .setNonceGroup(ng.getNonceGroup())
             .setNonce(ng.newNonce())
             .build();
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
-        return master.restoreSnapshot(controller, request);
+        return master.restoreSnapshot(rpcController, request);
       }
     });
 
@@ -2828,13 +2731,13 @@ public class HBaseAdmin implements Admin {
 
   @Override
   public List<SnapshotDescription> listSnapshots() throws IOException {
-    return executeCallable(new MasterCallable<List<SnapshotDescription>>(getConnection()) {
+    return executeCallable(new MasterCallable<List<SnapshotDescription>>(getConnection(),
+        getRpcControllerFactory()) {
       @Override
-      public List<SnapshotDescription> call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
+      protected List<SnapshotDescription> call(PayloadCarryingRpcController rpcController)
+      throws Exception {
         List<HBaseProtos.SnapshotDescription> snapshotsList = master
-            .getCompletedSnapshots(controller, GetCompletedSnapshotsRequest.newBuilder().build())
+            .getCompletedSnapshots(rpcController, GetCompletedSnapshotsRequest.newBuilder().build())
             .getSnapshotsList();
         List<SnapshotDescription> result = new ArrayList<SnapshotDescription>(snapshotsList.size());
         for (HBaseProtos.SnapshotDescription snapshot : snapshotsList) {
@@ -2897,14 +2800,11 @@ public class HBaseAdmin implements Admin {
     // make sure the snapshot is possibly valid
     TableName.isLegalFullyQualifiedTableName(Bytes.toBytes(snapshotName));
     // do the delete
-    executeCallable(new MasterCallable<Void>(getConnection()) {
+    executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
       @Override
-      public Void call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
-        master.deleteSnapshot(controller,
-          DeleteSnapshotRequest.newBuilder().
-              setSnapshot(
+      protected Void call(PayloadCarryingRpcController rpcController) throws Exception {
+        master.deleteSnapshot(rpcController,
+          DeleteSnapshotRequest.newBuilder().setSnapshot(
                 HBaseProtos.SnapshotDescription.newBuilder().setName(snapshotName).build())
               .build()
         );
@@ -2933,12 +2833,10 @@ public class HBaseAdmin implements Admin {
   }
 
   private void internalDeleteSnapshot(final SnapshotDescription snapshot) throws IOException {
-    executeCallable(new MasterCallable<Void>(getConnection()) {
+    executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
       @Override
-      public Void call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
-        this.master.deleteSnapshot(controller, DeleteSnapshotRequest.newBuilder()
+      protected Void call(PayloadCarryingRpcController rpcController) throws Exception {
+        this.master.deleteSnapshot(rpcController, DeleteSnapshotRequest.newBuilder()
           .setSnapshot(createHBaseProtosSnapshotDesc(snapshot)).build());
         return null;
       }
@@ -2967,11 +2865,10 @@ public class HBaseAdmin implements Admin {
 
   @Override
   public void setQuota(final QuotaSettings quota) throws IOException {
-    executeCallable(new MasterCallable<Void>(getConnection()) {
+    executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
       @Override
-      public Void call(int callTimeout) throws ServiceException {
+      protected Void call(PayloadCarryingRpcController rpcController) throws Exception {
         PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
         this.master.setQuota(controller, QuotaSettings.buildSetQuotaRequestProto(quota));
         return null;
       }
@@ -2989,8 +2886,8 @@ public class HBaseAdmin implements Admin {
   }
 
   static private <C extends RetryingCallable<V> & Closeable, V> V executeCallable(C callable,
-             RpcRetryingCallerFactory rpcCallerFactory, int operationTimeout,
-      int rpcTimeout) throws IOException {
+             RpcRetryingCallerFactory rpcCallerFactory, int operationTimeout, int rpcTimeout)
+  throws IOException {
     RpcRetryingCaller<V> caller = rpcCallerFactory.newCaller(rpcTimeout);
     try {
       return caller.callWithRetries(callable, operationTimeout);
@@ -3008,7 +2905,6 @@ public class HBaseAdmin implements Admin {
    * Simple {@link Abortable}, throwing RuntimeException on abort.
    */
   private static class ThrowableAbortable implements Abortable {
-
     @Override
     public void abort(String why, Throwable e) {
       throw new RuntimeException(why, e);
@@ -3026,13 +2922,16 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public void updateConfiguration(ServerName server) throws IOException {
-    try {
-      this.connection.getAdmin(server).updateConfiguration(null,
-        UpdateConfigurationRequest.getDefaultInstance());
-    } catch (ServiceException e) {
-      throw ProtobufUtil.getRemoteException(e);
-    }
+  public void updateConfiguration(final ServerName server) throws IOException {
+    final AdminService.BlockingInterface admin = this.connection.getAdmin(server);
+    Callable<Void> callable = new Callable<Void>() {
+      @Override
+      public Void call() throws Exception {
+        admin.updateConfiguration(null, UpdateConfigurationRequest.getDefaultInstance());
+        return null;
+      }
+    };
+    ProtobufUtil.call(callable);
   }
 
   @Override
@@ -3045,8 +2944,7 @@ public class HBaseAdmin implements Admin {
   @Override
   public int getMasterInfoPort() throws IOException {
     // TODO: Fix!  Reaching into internal implementation!!!!
-    ConnectionImplementation connection =
-        (ConnectionImplementation)this.connection;
+    ConnectionImplementation connection = (ConnectionImplementation)this.connection;
     ZooKeeperKeepAliveConnection zkw = connection.getKeepAliveZooKeeperWatcher();
     try {
       return MasterAddressTracker.getMasterInfoPort(zkw);
@@ -3057,8 +2955,7 @@ public class HBaseAdmin implements Admin {
 
   private ServerName getMasterAddress() throws IOException {
     // TODO: Fix!  Reaching into internal implementation!!!!
-    ConnectionImplementation connection =
-            (ConnectionImplementation)this.connection;
+    ConnectionImplementation connection = (ConnectionImplementation)this.connection;
     ZooKeeperKeepAliveConnection zkw = connection.getKeepAliveZooKeeperWatcher();
     try {
       return MasterAddressTracker.getMasterAddress(zkw);
@@ -3069,33 +2966,26 @@ public class HBaseAdmin implements Admin {
 
   @Override
   public long getLastMajorCompactionTimestamp(final TableName tableName) throws IOException {
-    return executeCallable(new MasterCallable<Long>(getConnection()) {
+    return executeCallable(new MasterCallable<Long>(getConnection(), getRpcControllerFactory()) {
       @Override
-      public Long call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
+      protected Long call(PayloadCarryingRpcController rpcController) throws Exception {
         MajorCompactionTimestampRequest req =
             MajorCompactionTimestampRequest.newBuilder()
                 .setTableName(ProtobufUtil.toProtoTableName(tableName)).build();
-        return master.getLastMajorCompactionTimestamp(controller, req).getCompactionTimestamp();
+        return master.getLastMajorCompactionTimestamp(rpcController, req).getCompactionTimestamp();
       }
     });
   }
 
   @Override
   public long getLastMajorCompactionTimestampForRegion(final byte[] regionName) throws IOException {
-    return executeCallable(new MasterCallable<Long>(getConnection()) {
+    return executeCallable(new MasterCallable<Long>(getConnection(), getRpcControllerFactory()) {
       @Override
-      public Long call(int callTimeout) throws ServiceException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setCallTimeout(callTimeout);
+      protected Long call(PayloadCarryingRpcController rpcController) throws Exception {
         MajorCompactionTimestampForRegionRequest req =
-            MajorCompactionTimestampForRegionRequest
-                .newBuilder()
-                .setRegion(
-                  RequestConverter
+            MajorCompactionTimestampForRegionRequest.newBuilder().setRegion(RequestConverter
                       .buildRegionSpecifier(RegionSpecifierType.REGION_NAME, regionName)).build();
-        return master.getLastMajorCompactionTimestampForRegion(controller, req)
+        return master.getLastMajorCompactionTimestampForRegion(rpcController, req)
             .getCompactionTimestamp();
       }
     });
@@ -3134,32 +3024,35 @@ public class HBaseAdmin implements Admin {
   @Override
   public void majorCompact(final TableName tableName, CompactType compactType)
           throws IOException, InterruptedException {
-      compact(tableName, null, true, compactType);
+    compact(tableName, null, true, compactType);
   }
 
   /**
    * {@inheritDoc}
    */
   @Override
-  public CompactionState getCompactionState(TableName tableName,
+  public CompactionState getCompactionState(final TableName tableName,
     CompactType compactType) throws IOException {
     AdminProtos.GetRegionInfoResponse.CompactionState state =
         AdminProtos.GetRegionInfoResponse.CompactionState.NONE;
     checkTableExists(tableName);
-    PayloadCarryingRpcController controller = rpcControllerFactory.newController();
+    final PayloadCarryingRpcController rpcController = rpcControllerFactory.newController();
     switch (compactType) {
       case MOB:
-        try {
-          ServerName master = getMasterAddress();
-          HRegionInfo info = getMobRegionInfo(tableName);
-          GetRegionInfoRequest request = RequestConverter.buildGetRegionInfoRequest(
-                  info.getRegionName(), true);
-          GetRegionInfoResponse response = this.connection.getAdmin(master)
-                  .getRegionInfo(controller, request);
-          state = response.getCompactionState();
-        } catch (ServiceException se) {
-          throw ProtobufUtil.getRemoteException(se);
-        }
+        final AdminProtos.AdminService.BlockingInterface masterAdmin =
+          this.connection.getAdmin(getMasterAddress());
+        Callable<AdminProtos.GetRegionInfoResponse.CompactionState> callable =
+            new Callable<AdminProtos.GetRegionInfoResponse.CompactionState>() {
+          @Override
+          public AdminProtos.GetRegionInfoResponse.CompactionState call() throws Exception {
+            HRegionInfo info = getMobRegionInfo(tableName);
+            GetRegionInfoRequest request = RequestConverter.buildGetRegionInfoRequest(
+                info.getRegionName(), true);
+            GetRegionInfoResponse response = masterAdmin.getRegionInfo(rpcController, request);
+            return response.getCompactionState();
+          }
+        };
+        state = ProtobufUtil.call(callable);
         break;
       case NORMAL:
       default:
@@ -3173,15 +3066,23 @@ public class HBaseAdmin implements Admin {
           } else {
             pairs = MetaTableAccessor.getTableRegionsAndLocations(connection, tableName);
           }
-          for (Pair<HRegionInfo, ServerName> pair : pairs) {
+          for (Pair<HRegionInfo, ServerName> pair: pairs) {
             if (pair.getFirst().isOffline()) continue;
             if (pair.getSecond() == null) continue;
+            final ServerName sn = pair.getSecond();
+            final byte [] regionName = pair.getFirst().getRegionName();
+            final AdminService.BlockingInterface snAdmin = this.connection.getAdmin(sn);
             try {
-              ServerName sn = pair.getSecond();
-              AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
-              GetRegionInfoRequest request = RequestConverter.buildGetRegionInfoRequest(
-                      pair.getFirst().getRegionName(), true);
-              GetRegionInfoResponse response = admin.getRegionInfo(controller, request);
+              Callable<GetRegionInfoResponse> regionInfoCallable =
+                  new Callable<GetRegionInfoResponse>() {
+                @Override
+                public GetRegionInfoResponse call() throws Exception {
+                  GetRegionInfoRequest request = RequestConverter.buildGetRegionInfoRequest(
+                      regionName, true);
+                  return snAdmin.getRegionInfo(rpcController, request);
+                }
+              };
+              GetRegionInfoResponse response = ProtobufUtil.call(regionInfoCallable);
               switch (response.getCompactionState()) {
                 case MAJOR_AND_MINOR:
                   return CompactionState.MAJOR_AND_MINOR;
@@ -3217,8 +3118,6 @@ public class HBaseAdmin implements Admin {
               }
             }
           }
-        } catch (ServiceException se) {
-          throw ProtobufUtil.getRemoteException(se);
         } finally {
           if (zookeeper != null) {
             zookeeper.close();
@@ -3283,12 +3182,11 @@ public class HBaseAdmin implements Admin {
     protected AbortProcedureResponse abortProcedureResult(
         final AbortProcedureRequest request) throws IOException {
       return admin.executeCallable(new MasterCallable<AbortProcedureResponse>(
-          admin.getConnection()) {
+          admin.getConnection(), admin.getRpcControllerFactory()) {
         @Override
-        public AbortProcedureResponse call(int callTimeout) throws ServiceException {
-          PayloadCarryingRpcController controller = admin.getRpcControllerFactory().newController();
-          controller.setCallTimeout(callTimeout);
-          return master.abortProcedure(controller, request);
+        protected AbortProcedureResponse call(PayloadCarr

<TRUNCATED>

[41/50] [abbrv] hbase git commit: HBASE-16379 [replication] Minor improvement to replication/copy_tables_desc.rb

Posted by sy...@apache.org.
HBASE-16379 [replication] Minor improvement to replication/copy_tables_desc.rb


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

Branch: refs/heads/hbase-12439
Commit: b3888eadf8ac081729385b434f8da626d02d031a
Parents: 45bb618
Author: Esteban Gutierrez <es...@apache.org>
Authored: Mon Aug 8 16:10:29 2016 -0700
Committer: Esteban Gutierrez <es...@apache.org>
Committed: Wed Aug 10 11:00:23 2016 -0700

----------------------------------------------------------------------
 bin/replication/copy_tables_desc.rb | 39 +++++++++++++++++++++++++++-----
 1 file changed, 33 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/b3888ead/bin/replication/copy_tables_desc.rb
----------------------------------------------------------------------
diff --git a/bin/replication/copy_tables_desc.rb b/bin/replication/copy_tables_desc.rb
index 8a6c670..a7cc3a3 100644
--- a/bin/replication/copy_tables_desc.rb
+++ b/bin/replication/copy_tables_desc.rb
@@ -27,7 +27,6 @@ include Java
 import org.apache.commons.logging.LogFactory
 import org.apache.hadoop.hbase.HBaseConfiguration
 import org.apache.hadoop.hbase.HConstants
-import org.apache.hadoop.hbase.EmptyWatcher
 import org.apache.hadoop.hbase.client.HBaseAdmin
 import org.apache.hadoop.hbase.HTableDescriptor
 import org.apache.hadoop.conf.Configuration
@@ -38,11 +37,32 @@ NAME = "copy_tables_desc"
 
 # Print usage for this script
 def usage
-  puts 'Usage: %s.rb master_zookeeper.quorum.peers:clientport:znode_parent slave_zookeeper.quorum.peers:clientport:znode_parent' % NAME
+  puts 'Usage: %s.rb master_zookeeper.quorum.peers:clientport:znode_parent slave_zookeeper.quorum.peers:clientport:znode_parent [table1,table2,table3,...]' % NAME
   exit!
 end
 
-if ARGV.size != 2
+def copy (src, dst, table)
+  # verify if table exists in source cluster
+  begin
+    t = src.getTableDescriptor(table.to_java_bytes)
+  rescue org.apache.hadoop.hbase.TableNotFoundException
+    puts "Source table \"%s\" doesn't exist, skipping." % table
+    return
+  end
+
+  # verify if table *doesn't* exists in the target cluster
+  begin
+    dst.createTable(t)
+  rescue org.apache.hadoop.hbase.TableExistsException
+    puts "Destination table \"%s\" exists in remote cluster, skipping." % table
+    return
+  end
+
+  puts "Schema for table \"%s\" was succesfully copied to remote cluster." % table
+end
+
+
+if ARGV.size < 2 || ARGV.size > 3
   usage
 end
 
@@ -52,6 +72,8 @@ parts1 = ARGV[0].split(":")
 
 parts2 = ARGV[1].split(":")
 
+parts3 = ARGV[2].split(",") unless ARGV[2].nil?
+
 c1 = HBaseConfiguration.create()
 c1.set(HConstants::ZOOKEEPER_QUORUM, parts1[0])
 c1.set("hbase.zookeeper.property.clientPort", parts1[1])
@@ -68,11 +90,16 @@ c2.set(HConstants::ZOOKEEPER_ZNODE_PARENT, parts2[2])
 connection2 = ConnectionFactory.createConnection(c2)
 admin2 = connection2.getAdmin()
 
-for t in admin1.listTables()
-  admin2.createTable(t)
+if parts3.nil?
+  admin1.listTableNames().each do |t|
+    copy(admin1, admin2, t.nameAsString())
+  end
+else
+  parts3.each do |t|
+    copy(admin1, admin2, t)
+  end
 end
 
-puts "All descriptions were copied"
 admin1.close()
 admin2.close()
 connection1.close()


[02/50] [abbrv] hbase git commit: HBASE-16355 Invalid hbase-common test-jar dependency scope

Posted by sy...@apache.org.
HBASE-16355 Invalid hbase-common test-jar dependency scope

hbase-client has hbase-common test-jar as dependency in compile scope, while it should be test scope instead.
This patch fixes the bug.

closes #12

Signed-off-by: Sean Busbey <bu...@apache.org>


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

Branch: refs/heads/hbase-12439
Commit: 544dc1e55851e42a24bd14eb9efb9b21c2447d6b
Parents: 6fdf3aa
Author: Stevo Slavi\u0107 <ss...@gmail.com>
Authored: Sun Apr 5 12:43:48 2015 +0200
Committer: Sean Busbey <bu...@apache.org>
Committed: Thu Aug 4 01:17:51 2016 -0500

----------------------------------------------------------------------
 hbase-client/pom.xml | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/544dc1e5/hbase-client/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-client/pom.xml b/hbase-client/pom.xml
index e74e0d5..7fc4672 100644
--- a/hbase-client/pom.xml
+++ b/hbase-client/pom.xml
@@ -130,6 +130,7 @@
       <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-common</artifactId>
       <type>test-jar</type>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hbase</groupId>


[26/50] [abbrv] hbase git commit: HBASE-12770 Don't transfer all the queued hlogs of a dead server to the same alive server

Posted by sy...@apache.org.
HBASE-12770 Don't transfer all the queued hlogs of a dead server to the same alive server

Signed-off-by: zhangduo <zh...@apache.org>


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

Branch: refs/heads/hbase-12439
Commit: e5f9df1e2394994323b4a5bfe2d7ba58aa669acd
Parents: 30d7eea
Author: Phil Yang <ud...@gmail.com>
Authored: Thu Jul 21 16:33:44 2016 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Aug 8 16:30:23 2016 +0800

----------------------------------------------------------------------
 .../hbase/replication/ReplicationQueues.java    |  25 +-
 .../replication/ReplicationQueuesZKImpl.java    | 263 ++++++++++---------
 .../TableBasedReplicationQueuesImpl.java        |  53 ++--
 .../regionserver/ReplicationSourceManager.java  |  26 +-
 .../replication/TestReplicationStateBasic.java  |  17 +-
 .../TestReplicationStateHBaseImpl.java          |  32 ++-
 .../TestReplicationSourceManager.java           |   8 +-
 .../TestReplicationSourceManagerZkImpl.java     |  28 +-
 8 files changed, 264 insertions(+), 188 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/e5f9df1e/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java
index 0de0cc8..0ae27d0 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java
@@ -19,10 +19,10 @@
 package org.apache.hadoop.hbase.replication;
 
 import java.util.List;
-import java.util.Map;
-import java.util.Set;
+import java.util.SortedSet;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.util.Pair;
 
 /**
  * This provides an interface for maintaining a region server's replication queues. These queues
@@ -94,12 +94,25 @@ public interface ReplicationQueues {
   List<String> getAllQueues();
 
   /**
-   * Take ownership for the set of queues belonging to a dead region server.
+   * Get queueIds from a dead region server, whose queues has not been claimed by other region
+   * servers.
+   * @return empty if the queue exists but no children, null if the queue does not exist.
+  */
+  List<String> getUnClaimedQueueIds(String regionserver);
+
+  /**
+   * Take ownership for the queue identified by queueId and belongs to a dead region server.
    * @param regionserver the id of the dead region server
-   * @return A Map of the queues that have been claimed, including a Set of WALs in
-   *         each queue. Returns an empty map if no queues were failed-over.
+   * @param queueId the id of the queue
+   * @return the new PeerId and A SortedSet of WALs in its queue, and null if no unclaimed queue.
+   */
+  Pair<String, SortedSet<String>> claimQueue(String regionserver, String queueId);
+
+  /**
+   * Remove the znode of region server if the queue is empty.
+   * @param regionserver
    */
-  Map<String, Set<String>> claimQueues(String regionserver);
+  void removeReplicatorIfQueueIsEmpty(String regionserver);
 
   /**
    * Get a list of all region servers that have outstanding replication queues. These servers could

http://git-wip-us.apache.org/repos/asf/hbase/blob/e5f9df1e/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
index 655aaae..c1e85cb 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
@@ -19,11 +19,9 @@
 package org.apache.hadoop.hbase.replication;
 
 import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
 import java.util.List;
-import java.util.Map;
-import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
 
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.logging.Log;
@@ -36,6 +34,7 @@ import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
@@ -179,21 +178,66 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
   }
 
   @Override
-  public Map<String, Set<String>> claimQueues(String regionserverZnode) {
-    Map<String, Set<String>> newQueues = new HashMap<>();
-    // check whether there is multi support. If yes, use it.
+  public List<String> getUnClaimedQueueIds(String regionserver) {
+    if (isThisOurRegionServer(regionserver)) {
+      return null;
+    }
+    String rsZnodePath = ZKUtil.joinZNode(this.queuesZNode, regionserver);
+    List<String> queues = null;
+    try {
+      queues = ZKUtil.listChildrenNoWatch(this.zookeeper, rsZnodePath);
+    } catch (KeeperException e) {
+      this.abortable.abort("Failed to getUnClaimedQueueIds for RS" + regionserver, e);
+    }
+    if (queues != null) {
+      queues.remove(RS_LOCK_ZNODE);
+    }
+    return queues;
+  }
+
+  @Override
+  public Pair<String, SortedSet<String>> claimQueue(String regionserver, String queueId) {
     if (conf.getBoolean(HConstants.ZOOKEEPER_USEMULTI, true)) {
-      LOG.info("Atomically moving " + regionserverZnode + "'s WALs to my queue");
-      newQueues = copyQueuesFromRSUsingMulti(regionserverZnode);
+      LOG.info("Atomically moving " + regionserver + "/" + queueId + "'s WALs to my queue");
+      return moveQueueUsingMulti(regionserver, queueId);
     } else {
-      LOG.info("Moving " + regionserverZnode + "'s wals to my queue");
-      if (!lockOtherRS(regionserverZnode)) {
-        return newQueues;
+      LOG.info("Moving " + regionserver + "/" + queueId + "'s wals to my queue");
+      if (!lockOtherRS(regionserver)) {
+        LOG.info("Can not take the lock now");
+        return null;
       }
-      newQueues = copyQueuesFromRS(regionserverZnode);
-      deleteAnotherRSQueues(regionserverZnode);
+      Pair<String, SortedSet<String>> newQueues;
+      try {
+        newQueues = copyQueueFromLockedRS(regionserver, queueId);
+        removeQueueFromLockedRS(regionserver, queueId);
+      } finally {
+        unlockOtherRS(regionserver);
+      }
+      return newQueues;
+    }
+  }
+
+  private void removeQueueFromLockedRS(String znode, String peerId) {
+    String nodePath = ZKUtil.joinZNode(this.queuesZNode, znode);
+    String peerPath = ZKUtil.joinZNode(nodePath, peerId);
+    try {
+      ZKUtil.deleteNodeRecursively(this.zookeeper, peerPath);
+    } catch (KeeperException e) {
+      LOG.warn("Remove copied queue failed", e);
+    }
+  }
+
+  @Override
+  public void removeReplicatorIfQueueIsEmpty(String regionserver) {
+    String rsPath = ZKUtil.joinZNode(this.queuesZNode, regionserver);
+    try {
+      List<String> list = ZKUtil.listChildrenNoWatch(this.zookeeper, rsPath);
+      if (list != null && list.size() == 0){
+        ZKUtil.deleteNode(this.zookeeper, rsPath);
+      }
+    } catch (KeeperException e) {
+      LOG.warn("Got error while removing replicator", e);
     }
-    return newQueues;
   }
 
   @Override
@@ -276,36 +320,13 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
     return ZKUtil.checkExists(zookeeper, getLockZNode(znode)) >= 0;
   }
 
-  /**
-   * Delete all the replication queues for a given region server.
-   * @param regionserverZnode The znode of the region server to delete.
-   */
-  private void deleteAnotherRSQueues(String regionserverZnode) {
-    String fullpath = ZKUtil.joinZNode(this.queuesZNode, regionserverZnode);
+  private void unlockOtherRS(String znode){
+    String parent = ZKUtil.joinZNode(this.queuesZNode, znode);
+    String p = ZKUtil.joinZNode(parent, RS_LOCK_ZNODE);
     try {
-      List<String> clusters = ZKUtil.listChildrenNoWatch(this.zookeeper, fullpath);
-      for (String cluster : clusters) {
-        // No need to delete, it will be deleted later.
-        if (cluster.equals(RS_LOCK_ZNODE)) {
-          continue;
-        }
-        String fullClusterPath = ZKUtil.joinZNode(fullpath, cluster);
-        ZKUtil.deleteNodeRecursively(this.zookeeper, fullClusterPath);
-      }
-      // Finish cleaning up
-      ZKUtil.deleteNodeRecursively(this.zookeeper, fullpath);
+      ZKUtil.deleteNode(this.zookeeper, p);
     } catch (KeeperException e) {
-      if (e instanceof KeeperException.NoNodeException
-          || e instanceof KeeperException.NotEmptyException) {
-        // Testing a special case where another region server was able to
-        // create a lock just after we deleted it, but then was also able to
-        // delete the RS znode before us or its lock znode is still there.
-        if (e.getPath().equals(fullpath)) {
-          return;
-        }
-      }
-      this.abortable.abort("Failed to delete replication queues for region server: "
-          + regionserverZnode, e);
+      this.abortable.abort("Remove lock failed", e);
     }
   }
 
@@ -313,38 +334,30 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
    * It "atomically" copies all the wals queues from another region server and returns them all
    * sorted per peer cluster (appended with the dead server's znode).
    * @param znode pertaining to the region server to copy the queues from
-   * @return WAL queues sorted per peer cluster
    */
-  private Map<String, Set<String>> copyQueuesFromRSUsingMulti(String znode) {
-    Map<String, Set<String>> queues = new HashMap<>();
-    // hbase/replication/rs/deadrs
-    String deadRSZnodePath = ZKUtil.joinZNode(this.queuesZNode, znode);
-    List<String> peerIdsToProcess = null;
-    List<ZKUtilOp> listOfOps = new ArrayList<ZKUtil.ZKUtilOp>();
+  private Pair<String, SortedSet<String>> moveQueueUsingMulti(String znode, String peerId) {
     try {
-      peerIdsToProcess = ZKUtil.listChildrenNoWatch(this.zookeeper, deadRSZnodePath);
-      if (peerIdsToProcess == null) return queues; // node already processed
-      for (String peerId : peerIdsToProcess) {
-        ReplicationQueueInfo replicationQueueInfo = new ReplicationQueueInfo(peerId);
-        if (!peerExists(replicationQueueInfo.getPeerId())) {
-          // the orphaned queues must be moved, otherwise the delete op of dead rs will fail,
-          // this will cause the whole multi op fail.
-          // NodeFailoverWorker will skip the orphaned queues.
-          LOG.warn("Peer " + peerId
-              + " didn't exist, will move its queue to avoid the failure of multi op");
-        }
-        String newPeerId = peerId + "-" + znode;
-        String newPeerZnode = ZKUtil.joinZNode(this.myQueuesZnode, newPeerId);
-        // check the logs queue for the old peer cluster
-        String oldClusterZnode = ZKUtil.joinZNode(deadRSZnodePath, peerId);
-        List<String> wals = ZKUtil.listChildrenNoWatch(this.zookeeper, oldClusterZnode);
-        if (wals == null || wals.size() == 0) {
-          listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldClusterZnode));
-          continue; // empty log queue.
-        }
+      // hbase/replication/rs/deadrs
+      String deadRSZnodePath = ZKUtil.joinZNode(this.queuesZNode, znode);
+      List<ZKUtilOp> listOfOps = new ArrayList<>();
+      ReplicationQueueInfo replicationQueueInfo = new ReplicationQueueInfo(peerId);
+      if (!peerExists(replicationQueueInfo.getPeerId())) {
+        // the orphaned queues must be moved, otherwise the delete op of dead rs will fail,
+        // this will cause the whole multi op fail.
+        // NodeFailoverWorker will skip the orphaned queues.
+        LOG.warn("Peer " + peerId +
+            " didn't exist, will move its queue to avoid the failure of multi op");
+      }
+      String newPeerId = peerId + "-" + znode;
+      String newPeerZnode = ZKUtil.joinZNode(this.myQueuesZnode, newPeerId);
+      // check the logs queue for the old peer cluster
+      String oldClusterZnode = ZKUtil.joinZNode(deadRSZnodePath, peerId);
+      List<String> wals = ZKUtil.listChildrenNoWatch(this.zookeeper, oldClusterZnode);
+      SortedSet<String> logQueue = new TreeSet<>();
+      if (wals == null || wals.size() == 0) {
+        listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldClusterZnode));
+      } else {
         // create the new cluster znode
-        Set<String> logQueue = new HashSet<String>();
-        queues.put(newPeerId, logQueue);
         ZKUtilOp op = ZKUtilOp.createAndFailSilent(newPeerZnode, HConstants.EMPTY_BYTE_ARRAY);
         listOfOps.add(op);
         // get the offset of the logs and set it to new znodes
@@ -354,98 +367,86 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
           LOG.debug("Creating " + wal + " with data " + Bytes.toString(logOffset));
           String newLogZnode = ZKUtil.joinZNode(newPeerZnode, wal);
           listOfOps.add(ZKUtilOp.createAndFailSilent(newLogZnode, logOffset));
-          // add ops for deleting
           listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldWalZnode));
           logQueue.add(wal);
         }
         // add delete op for peer
         listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldClusterZnode));
+
+        if (LOG.isTraceEnabled())
+          LOG.trace(" The multi list size is: " + listOfOps.size());
       }
-      // add delete op for dead rs, this will update the cversion of the parent.
-      // The reader will make optimistic locking with this to get a consistent
-      // snapshot
-      listOfOps.add(ZKUtilOp.deleteNodeFailSilent(deadRSZnodePath));
-      if (LOG.isTraceEnabled()) LOG.trace(" The multi list size is: " + listOfOps.size());
       ZKUtil.multiOrSequential(this.zookeeper, listOfOps, false);
-      if (LOG.isTraceEnabled()) LOG.trace("Atomically moved the dead regionserver logs. ");
+      if (LOG.isTraceEnabled())
+        LOG.trace("Atomically moved the dead regionserver logs. ");
+      return new Pair<>(newPeerId, logQueue);
     } catch (KeeperException e) {
       // Multi call failed; it looks like some other regionserver took away the logs.
       LOG.warn("Got exception in copyQueuesFromRSUsingMulti: ", e);
-      queues.clear();
     } catch (InterruptedException e) {
       LOG.warn("Got exception in copyQueuesFromRSUsingMulti: ", e);
-      queues.clear();
       Thread.currentThread().interrupt();
     }
-    return queues;
+    return null;
   }
 
   /**
-   * This methods copies all the wals queues from another region server and returns them all sorted
+   * This methods moves all the wals queues from another region server and returns them all sorted
    * per peer cluster (appended with the dead server's znode)
    * @param znode server names to copy
-   * @return all wals for all peers of that cluster, null if an error occurred
+   * @return all wals for the peer of that cluster, null if an error occurred
    */
-  private Map<String, Set<String>> copyQueuesFromRS(String znode) {
+  private Pair<String, SortedSet<String>> copyQueueFromLockedRS(String znode, String peerId) {
     // TODO this method isn't atomic enough, we could start copying and then
     // TODO fail for some reason and we would end up with znodes we don't want.
-    Map<String, Set<String>> queues = new HashMap<>();
     try {
       String nodePath = ZKUtil.joinZNode(this.queuesZNode, znode);
-      List<String> clusters = ZKUtil.listChildrenNoWatch(this.zookeeper, nodePath);
-      // We have a lock znode in there, it will count as one.
-      if (clusters == null || clusters.size() <= 1) {
-        return queues;
+      ReplicationQueueInfo replicationQueueInfo = new ReplicationQueueInfo(peerId);
+      String clusterPath = ZKUtil.joinZNode(nodePath, peerId);
+      if (!peerExists(replicationQueueInfo.getPeerId())) {
+        LOG.warn("Peer " + peerId + " didn't exist, skipping the replay");
+        // Protection against moving orphaned queues
+        return null;
       }
-      // The lock isn't a peer cluster, remove it
-      clusters.remove(RS_LOCK_ZNODE);
-      for (String cluster : clusters) {
-        ReplicationQueueInfo replicationQueueInfo = new ReplicationQueueInfo(cluster);
-        if (!peerExists(replicationQueueInfo.getPeerId())) {
-          LOG.warn("Peer " + cluster + " didn't exist, skipping the replay");
-          // Protection against moving orphaned queues
-          continue;
-        }
-        // We add the name of the recovered RS to the new znode, we can even
-        // do that for queues that were recovered 10 times giving a znode like
-        // number-startcode-number-otherstartcode-number-anotherstartcode-etc
-        String newCluster = cluster + "-" + znode;
-        String newClusterZnode = ZKUtil.joinZNode(this.myQueuesZnode, newCluster);
-        String clusterPath = ZKUtil.joinZNode(nodePath, cluster);
-        List<String> wals = ZKUtil.listChildrenNoWatch(this.zookeeper, clusterPath);
-        // That region server didn't have anything to replicate for this cluster
-        if (wals == null || wals.size() == 0) {
-          continue;
-        }
-        ZKUtil.createNodeIfNotExistsAndWatch(this.zookeeper, newClusterZnode,
+      // We add the name of the recovered RS to the new znode, we can even
+      // do that for queues that were recovered 10 times giving a znode like
+      // number-startcode-number-otherstartcode-number-anotherstartcode-etc
+      String newCluster = peerId + "-" + znode;
+      String newClusterZnode = ZKUtil.joinZNode(this.myQueuesZnode, newCluster);
+
+      List<String> wals = ZKUtil.listChildrenNoWatch(this.zookeeper, clusterPath);
+      // That region server didn't have anything to replicate for this cluster
+      if (wals == null || wals.size() == 0) {
+        return null;
+      }
+      ZKUtil.createNodeIfNotExistsAndWatch(this.zookeeper, newClusterZnode,
           HConstants.EMPTY_BYTE_ARRAY);
-        Set<String> logQueue = new HashSet<String>();
-        queues.put(newCluster, logQueue);
-        for (String wal : wals) {
-          String z = ZKUtil.joinZNode(clusterPath, wal);
-          byte[] positionBytes = ZKUtil.getData(this.zookeeper, z);
-          long position = 0;
-          try {
-            position = ZKUtil.parseWALPositionFrom(positionBytes);
-          } catch (DeserializationException e) {
-            LOG.warn("Failed parse of wal position from the following znode: " + z
-                + ", Exception: " + e);
-          }
-          LOG.debug("Creating " + wal + " with data " + position);
-          String child = ZKUtil.joinZNode(newClusterZnode, wal);
-          // Position doesn't actually change, we are just deserializing it for
-          // logging, so just use the already serialized version
-          ZKUtil.createAndWatch(this.zookeeper, child, positionBytes);
-          logQueue.add(wal);
+      SortedSet<String> logQueue = new TreeSet<>();
+      for (String wal : wals) {
+        String z = ZKUtil.joinZNode(clusterPath, wal);
+        byte[] positionBytes = ZKUtil.getData(this.zookeeper, z);
+        long position = 0;
+        try {
+          position = ZKUtil.parseWALPositionFrom(positionBytes);
+        } catch (DeserializationException e) {
+          LOG.warn("Failed parse of wal position from the following znode: " + z
+              + ", Exception: " + e);
         }
+        LOG.debug("Creating " + wal + " with data " + position);
+        String child = ZKUtil.joinZNode(newClusterZnode, wal);
+        // Position doesn't actually change, we are just deserializing it for
+        // logging, so just use the already serialized version
+        ZKUtil.createNodeIfNotExistsAndWatch(this.zookeeper, child, positionBytes);
+        logQueue.add(wal);
       }
+      return new Pair<>(newCluster, logQueue);
     } catch (KeeperException e) {
-      this.abortable.abort("Copy queues from rs", e);
+      LOG.warn("Got exception in copyQueueFromLockedRS: ", e);
     } catch (InterruptedException e) {
       LOG.warn(e);
       Thread.currentThread().interrupt();
     }
-    return queues;
+    return null;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/e5f9df1e/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/TableBasedReplicationQueuesImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/TableBasedReplicationQueuesImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/TableBasedReplicationQueuesImpl.java
index 3ee6fde..28b9bdf 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/TableBasedReplicationQueuesImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/TableBasedReplicationQueuesImpl.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.filter.CompareFilter;
 import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.zookeeper.KeeperException;
 
@@ -48,6 +49,8 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
 
 /**
  * This class provides an implementation of the ReplicationQueues interface using an HBase table
@@ -227,31 +230,54 @@ public class TableBasedReplicationQueuesImpl extends ReplicationTableBase
     return getAllQueues(serverName);
   }
 
+  @Override public List<String> getUnClaimedQueueIds(String regionserver) {
+    if (isThisOurRegionServer(regionserver)) {
+      return null;
+    }
+    try (ResultScanner queuesToClaim = getQueuesBelongingToServer(regionserver)) {
+      List<String> res = new ArrayList<>();
+      for (Result queue : queuesToClaim) {
+        String rowKey = Bytes.toString(queue.getRow());
+        res.add(rowKey);
+      }
+      return res.isEmpty() ? null : res;
+    } catch (IOException e) {
+      String errMsg = "Failed getUnClaimedQueueIds";
+      abortable.abort(errMsg, e);
+    }
+    return null;
+  }
+
+  @Override public void removeReplicatorIfQueueIsEmpty(String regionserver) {
+    // Do nothing here
+  }
+
   @Override
-  public Map<String, Set<String>> claimQueues(String regionserver) {
-    Map<String, Set<String>> queues = new HashMap<>();
+  public Pair<String, SortedSet<String>> claimQueue(String regionserver, String queueId) {
     if (isThisOurRegionServer(regionserver)) {
-      return queues;
+      return null;
     }
-    ResultScanner queuesToClaim = null;
-    try {
-      queuesToClaim = getQueuesBelongingToServer(regionserver);
+
+    try (ResultScanner queuesToClaim = getQueuesBelongingToServer(regionserver)){
       for (Result queue : queuesToClaim) {
+        String rowKey = Bytes.toString(queue.getRow());
+        if (!rowKey.equals(queueId)){
+          continue;
+        }
         if (attemptToClaimQueue(queue, regionserver)) {
-          String rowKey = Bytes.toString(queue.getRow());
           ReplicationQueueInfo replicationQueueInfo = new ReplicationQueueInfo(rowKey);
           if (replicationState.peerExists(replicationQueueInfo.getPeerId())) {
-            Set<String> sortedLogs = new HashSet<String>();
+            SortedSet<String> sortedLogs = new TreeSet<>();
             List<String> logs = getLogsInQueue(queue.getRow());
             for (String log : logs) {
               sortedLogs.add(log);
             }
-            queues.put(rowKey, sortedLogs);
             LOG.info(serverName + " has claimed queue " + rowKey + " from " + regionserver);
+            return new Pair<>(rowKey, sortedLogs);
           } else {
             // Delete orphaned queues
             removeQueue(Bytes.toString(queue.getRow()));
-            LOG.info(serverName + " has deleted abandoned queue " + rowKey + " from " +
+            LOG.info(serverName + " has deleted abandoned queue " + queueId + " from " +
               regionserver);
           }
         }
@@ -259,13 +285,8 @@ public class TableBasedReplicationQueuesImpl extends ReplicationTableBase
     } catch (IOException | KeeperException e) {
       String errMsg = "Failed claiming queues for regionserver=" + regionserver;
       abortable.abort(errMsg, e);
-      queues.clear();
-    } finally {
-      if (queuesToClaim != null) {
-        queuesToClaim.close();
-      }
     }
-    return queues;
+    return null;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/e5f9df1e/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
index 586aace..3cb7a84 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
@@ -63,6 +63,7 @@ import org.apache.hadoop.hbase.replication.ReplicationPeers;
 import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
 import org.apache.hadoop.hbase.replication.ReplicationQueues;
 import org.apache.hadoop.hbase.replication.ReplicationTracker;
+import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
 
 /**
@@ -647,10 +648,27 @@ public class ReplicationSourceManager implements ReplicationListener {
         LOG.info("Not transferring queue since we are shutting down");
         return;
       }
-      Map<String, Set<String>> newQueues = null;
-
-      newQueues = this.rq.claimQueues(rsZnode);
-
+      Map<String, Set<String>> newQueues = new HashMap<>();
+      List<String> peers = rq.getUnClaimedQueueIds(rsZnode);
+      while (peers != null && !peers.isEmpty()) {
+        Pair<String, SortedSet<String>> peer = this.rq.claimQueue(rsZnode,
+            peers.get(rand.nextInt(peers.size())));
+        long sleep = sleepBeforeFailover/2;
+        if (peer != null) {
+          newQueues.put(peer.getFirst(), peer.getSecond());
+          sleep = sleepBeforeFailover;
+        }
+        try {
+          Thread.sleep(sleep);
+        } catch (InterruptedException e) {
+          LOG.warn("Interrupted while waiting before transferring a queue.");
+          Thread.currentThread().interrupt();
+        }
+        peers = rq.getUnClaimedQueueIds(rsZnode);
+      }
+      if (peers != null) {
+        rq.removeReplicatorIfQueueIsEmpty(rsZnode);
+      }
       // Copying over the failed queue is completed.
       if (newQueues.isEmpty()) {
         // We either didn't get the lock or the failed region server didn't have any outstanding

http://git-wip-us.apache.org/repos/asf/hbase/blob/e5f9df1e/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
index 933c621..fcab105 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
@@ -22,8 +22,6 @@ import static org.junit.Assert.*;
 
 import java.util.ArrayList;
 import java.util.List;
-import java.util.Map;
-import java.util.Set;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -124,7 +122,8 @@ public abstract class TestReplicationStateBasic {
     assertEquals(0, rq1.getAllQueues().size());
     assertEquals(0, rq1.getLogPosition("bogus", "bogus"));
     assertNull(rq1.getLogsInQueue("bogus"));
-    assertEquals(0, rq1.claimQueues(ServerName.valueOf("bogus", 1234, -1L).toString()).size());
+    assertNull(rq1.getUnClaimedQueueIds(
+        ServerName.valueOf("bogus", 1234, -1L).toString()));
 
     rq1.setLogPosition("bogus", "bogus", 5L);
 
@@ -143,15 +142,21 @@ public abstract class TestReplicationStateBasic {
     assertEquals(1, rq2.getAllQueues().size());
     assertEquals(5, rq3.getAllQueues().size());
 
-    assertEquals(0, rq3.claimQueues(server1).size());
+    assertEquals(0, rq3.getUnClaimedQueueIds(server1).size());
+    rq3.removeReplicatorIfQueueIsEmpty(server1);
     assertEquals(2, rq3.getListOfReplicators().size());
 
-    Map<String, Set<String>> queues = rq2.claimQueues(server3);
+    List<String> queues = rq2.getUnClaimedQueueIds(server3);
     assertEquals(5, queues.size());
+    for(String queue: queues) {
+      rq2.claimQueue(server3, queue);
+    }
+    rq2.removeReplicatorIfQueueIsEmpty(server3);
     assertEquals(1, rq2.getListOfReplicators().size());
 
     // Try to claim our own queues
-    assertEquals(0, rq2.claimQueues(server2).size());
+    assertNull(rq2.getUnClaimedQueueIds(server2));
+    rq2.removeReplicatorIfQueueIsEmpty(server2);
 
     assertEquals(6, rq2.getAllQueues().size());
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/e5f9df1e/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateHBaseImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateHBaseImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateHBaseImpl.java
index 7ec6df8..35c4e24 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateHBaseImpl.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateHBaseImpl.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.zookeeper.KeeperException;
 import org.junit.After;
 import org.junit.AfterClass;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -40,8 +41,6 @@ import org.junit.experimental.categories.Category;
 
 import java.io.IOException;
 import java.util.List;
-import java.util.Map;
-import java.util.Set;
 
 import static junit.framework.TestCase.assertNull;
 import static org.junit.Assert.assertEquals;
@@ -267,18 +266,19 @@ public class TestReplicationStateHBaseImpl {
     }
     try {
       // Test claiming queues
-      Map<String, Set<String>> claimedQueuesFromRq2 = rq1.claimQueues(server2);
+      List<String> claimedQueuesFromRq2 = rq1.getUnClaimedQueueIds(server2);
       // Check to make sure that list of peers with outstanding queues is decremented by one
       // after claimQueues
+      // Check to make sure that we claimed the proper number of queues
+      assertEquals(2, claimedQueuesFromRq2.size());
+      assertTrue(claimedQueuesFromRq2.contains("Queue1-" + server2));
+      assertTrue(claimedQueuesFromRq2.contains("Queue2-" + server2));
+      assertEquals(2, rq1.claimQueue(server2, "Queue1-" + server2).getSecond().size());
+      assertEquals(1, rq1.claimQueue(server2, "Queue2-" + server2).getSecond().size());
+      rq1.removeReplicatorIfQueueIsEmpty(server2);
       assertEquals(rq1.getListOfReplicators().size(), 2);
       assertEquals(rq2.getListOfReplicators().size(), 2);
       assertEquals(rq3.getListOfReplicators().size(), 2);
-      // Check to make sure that we claimed the proper number of queues
-      assertEquals(2, claimedQueuesFromRq2.size());
-      assertTrue(claimedQueuesFromRq2.containsKey("Queue1-" + server2));
-      assertTrue(claimedQueuesFromRq2.containsKey("Queue2-" + server2));
-      assertEquals(2, claimedQueuesFromRq2.get("Queue1-" + server2).size());
-      assertEquals(1, claimedQueuesFromRq2.get("Queue2-" + server2).size());
       assertEquals(5, rq1.getAllQueues().size());
       // Check that all the logs in the other queue were claimed
       assertEquals(2, rq1.getLogsInQueue("Queue1-" + server2).size());
@@ -294,7 +294,11 @@ public class TestReplicationStateHBaseImpl {
       rq1.addLog("UnclaimableQueue", "WALLogFile1.1");
       rq1.addLog("UnclaimableQueue", "WALLogFile1.2");
       assertEquals(6, rq1.getAllQueues().size());
-      Map<String, Set<String>> claimedQueuesFromRq1 = rq3.claimQueues(server1);
+      List<String> claimedQueuesFromRq1 = rq3.getUnClaimedQueueIds(server1);
+      for(String queue : claimedQueuesFromRq1) {
+        rq3.claimQueue(server1, queue);
+      }
+      rq3.removeReplicatorIfQueueIsEmpty(server1);
       assertEquals(rq1.getListOfReplicators().size(), 1);
       assertEquals(rq2.getListOfReplicators().size(), 1);
       assertEquals(rq3.getListOfReplicators().size(), 1);
@@ -302,12 +306,12 @@ public class TestReplicationStateHBaseImpl {
       // Replication Peers
       assertEquals(6, rq3.getAllQueues().size());
       // Test claiming non-existing queues
-      Map<String, Set<String>> noQueues = rq3.claimQueues("NotARealServer");
-      assertEquals(0, noQueues.size());
+      List<String> noQueues = rq3.getUnClaimedQueueIds("NotARealServer");
+      assertNull(noQueues);
       assertEquals(6, rq3.getAllQueues().size());
       // Test claiming own queues
-      noQueues = rq3.claimQueues(server3);
-      assertEquals(0, noQueues.size());
+      noQueues = rq3.getUnClaimedQueueIds(server3);
+      Assert.assertNull(noQueues);
       assertEquals(6, rq3.getAllQueues().size());
       // Check that rq3 still remain on list of replicators
       assertEquals(1, rq3.getListOfReplicators().size());

http://git-wip-us.apache.org/repos/asf/hbase/blob/e5f9df1e/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
index 7696e95..4ee783d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
@@ -75,6 +75,7 @@ import org.apache.hadoop.hbase.testclassification.ReplicationTests;
 import org.apache.hadoop.hbase.util.ByteStringer;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.wal.WALFactory;
 import org.apache.hadoop.hbase.wal.WALKey;
@@ -517,7 +518,12 @@ public abstract class TestReplicationSourceManager {
     @Override
     public void run() {
       try {
-        logZnodesMap = rq.claimQueues(deadRsZnode);
+        logZnodesMap = new HashMap<>();
+        List<String> queues = rq.getUnClaimedQueueIds(deadRsZnode);
+        for(String queue:queues){
+          Pair<String, SortedSet<String>> pair = rq.claimQueue(deadRsZnode, queue);
+          logZnodesMap.put(pair.getFirst(), pair.getSecond());
+        }
         server.abort("Done with testing", null);
       } catch (Exception e) {
         LOG.error("Got exception while running NodeFailoverWorker", e);

http://git-wip-us.apache.org/repos/asf/hbase/blob/e5f9df1e/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManagerZkImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManagerZkImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManagerZkImpl.java
index 75ed835..a9d0766 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManagerZkImpl.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManagerZkImpl.java
@@ -36,8 +36,6 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import java.util.List;
-import java.util.Map;
-import java.util.Set;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
@@ -87,22 +85,28 @@ public class TestReplicationSourceManagerZkImpl extends TestReplicationSourceMan
       ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(s1.getConfiguration(), s1,
         s1.getZooKeeper()));
     rq1.init(s1.getServerName().toString());
-    Map<String, Set<String>> testMap =
-      rq1.claimQueues(server.getServerName().getServerName());
+    String serverName = server.getServerName().getServerName();
+    List<String> unclaimed = rq1.getUnClaimedQueueIds(serverName);
+    rq1.claimQueue(serverName, unclaimed.get(0)).getSecond();
+    rq1.removeReplicatorIfQueueIsEmpty(unclaimed.get(0));
     ReplicationQueues rq2 =
       ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(s2.getConfiguration(), s2,
         s2.getZooKeeper()));
     rq2.init(s2.getServerName().toString());
-    testMap = rq2.claimQueues(s1.getServerName().getServerName());
+    serverName = s1.getServerName().getServerName();
+    unclaimed = rq2.getUnClaimedQueueIds(serverName);
+    rq2.claimQueue(serverName, unclaimed.get(0)).getSecond();
+    rq2.removeReplicatorIfQueueIsEmpty(unclaimed.get(0));
     ReplicationQueues rq3 =
       ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(s3.getConfiguration(), s3,
         s3.getZooKeeper()));
     rq3.init(s3.getServerName().toString());
-    testMap = rq3.claimQueues(s2.getServerName().getServerName());
-
-    ReplicationQueueInfo replicationQueueInfo = new ReplicationQueueInfo(testMap.keySet().iterator().next());
+    serverName = s2.getServerName().getServerName();
+    unclaimed = rq3.getUnClaimedQueueIds(serverName);
+    String queue3 = rq3.claimQueue(serverName, unclaimed.get(0)).getFirst();
+    rq3.removeReplicatorIfQueueIsEmpty(unclaimed.get(0));
+    ReplicationQueueInfo replicationQueueInfo = new ReplicationQueueInfo(queue3);
     List<String> result = replicationQueueInfo.getDeadRegionServers();
-
     // verify
     assertTrue(result.contains(server.getServerName().getServerName()));
     assertTrue(result.contains(s1.getServerName().getServerName()));
@@ -137,7 +141,11 @@ public class TestReplicationSourceManagerZkImpl extends TestReplicationSourceMan
         new ReplicationQueuesClientArguments(s1.getConfiguration(), s1, s1.getZooKeeper()));
 
     int v0 = client.getQueuesZNodeCversion();
-    rq1.claimQueues(s0.getServerName().getServerName());
+    List<String> queues = rq1.getUnClaimedQueueIds(s0.getServerName().getServerName());
+    for(String queue : queues) {
+      rq1.claimQueue(s0.getServerName().getServerName(), queue);
+    }
+    rq1.removeReplicatorIfQueueIsEmpty(s0.getServerName().getServerName());
     int v1 = client.getQueuesZNodeCversion();
     // cversion should increase by 1 since a child node is deleted
     assertEquals(v0 + 1, v1);


[44/50] [abbrv] hbase git commit: HBASE-16389 Thread leak in CoprocessorHost#getTable(TableName) API (Ankit Singhal)

Posted by sy...@apache.org.
HBASE-16389 Thread leak in CoprocessorHost#getTable(TableName) API (Ankit Singhal)


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

Branch: refs/heads/hbase-12439
Commit: 9e116e048b91ee2fef056e4140cb4091a7fd8425
Parents: 7e32510
Author: tedyu <yu...@gmail.com>
Authored: Wed Aug 10 19:58:13 2016 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Wed Aug 10 19:58:13 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/9e116e04/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java
index 37c344b..993fa16 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java
@@ -540,7 +540,7 @@ public abstract class CoprocessorHost<E extends CoprocessorEnvironment> {
      */
     @Override
     public Table getTable(TableName tableName) throws IOException {
-      return this.getTable(tableName, HTable.getDefaultExecutor(getConfiguration()));
+      return this.getTable(tableName, null);
     }
 
     /**


[31/50] [abbrv] hbase git commit: HBASE-9899 for idempotent operation dups, return the result instead of throwing conflict exception (Guanghao Zhang): ADDENDUM

Posted by sy...@apache.org.
HBASE-9899 for idempotent operation dups, return the result instead of throwing conflict exception (Guanghao Zhang): ADDENDUM


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

Branch: refs/heads/hbase-12439
Commit: 46defe8e31278f91b5b92c24744b1839da564f27
Parents: 5cadcd5
Author: stack <st...@apache.org>
Authored: Tue Aug 9 07:09:44 2016 -0700
Committer: stack <st...@apache.org>
Committed: Tue Aug 9 07:09:44 2016 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hbase/regionserver/RSRpcServices.java   | 10 ++++++----
 1 file changed, 6 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/46defe8e/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index d87ada4..681b1dc 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -620,12 +620,13 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
       boolean canProceed = startNonceOperation(mutation, nonceGroup);
       boolean success = false;
       try {
+        long nonce = mutation.hasNonce() ? mutation.getNonce() : HConstants.NO_NONCE;
         if (canProceed) {
-          r = region.append(append, nonceGroup, mutation.getNonce());
+          r = region.append(append, nonceGroup, nonce);
         } else {
           // convert duplicate append to get
           List<Cell> results = region.get(ProtobufUtil.toGet(mutation, cellScanner), false,
-            nonceGroup, mutation.getNonce());
+            nonceGroup, nonce);
           r = Result.create(results);
         }
         success = true;
@@ -667,12 +668,13 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
       boolean canProceed = startNonceOperation(mutation, nonceGroup);
       boolean success = false;
       try {
+        long nonce = mutation.hasNonce() ? mutation.getNonce() : HConstants.NO_NONCE;
         if (canProceed) {
-          r = region.increment(increment, nonceGroup, mutation.getNonce());
+          r = region.increment(increment, nonceGroup, nonce);
         } else {
           // convert duplicate increment to get
           List<Cell> results = region.get(ProtobufUtil.toGet(mutation, cells), false, nonceGroup,
-            mutation.getNonce());
+            nonce);
           r = Result.create(results);
         }
         success = true;


[32/50] [abbrv] hbase git commit: HBASE-16310 Revisit the logic of filterRowKey for Filters (Ram)

Posted by sy...@apache.org.
HBASE-16310 Revisit the logic of filterRowKey for Filters (Ram)


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

Branch: refs/heads/hbase-12439
Commit: 2d203e6053defb52cd3629d49ae6d54460c658dd
Parents: 46defe8
Author: Ramkrishna <ra...@intel.com>
Authored: Wed Aug 10 13:54:25 2016 +0530
Committer: Ramkrishna <ra...@intel.com>
Committed: Wed Aug 10 13:54:25 2016 +0530

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java  | 1 +
 .../src/main/java/org/apache/hadoop/hbase/filter/Filter.java       | 2 ++
 .../src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java   | 2 ++
 .../main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java    | 2 ++
 .../java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java   | 1 +
 .../java/org/apache/hadoop/hbase/filter/MultiRowRangeFilter.java   | 1 +
 .../src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java   | 1 +
 .../src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java | 1 +
 .../main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java | 1 +
 9 files changed, 12 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/2d203e60/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java
index fd65130..c747b00 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java
@@ -55,6 +55,7 @@ public class ColumnCountGetFilter extends FilterBase {
   @Override
   public boolean filterRowKey(Cell cell) throws IOException {
     // Impl in FilterBase might do unnecessary copy for Off heap backed Cells.
+    if (filterAllRemaining()) return true;
     return false;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/2d203e60/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java
index 22ca8ac..59aa855 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java
@@ -87,6 +87,8 @@ public abstract class Filter {
   /**
    * Filters a row based on the row key. If this returns true, the entire row will be excluded. If
    * false, each KeyValue in the row will be passed to {@link #filterKeyValue(Cell)} below.
+   * If {@link #filterAllRemaining()} returns true, then {@link #filterRowKey(Cell)} should
+   * also return true.
    *
    * Concrete implementers can signal a failure condition in their code by throwing an
    * {@link IOException}.

http://git-wip-us.apache.org/repos/asf/hbase/blob/2d203e60/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java
index fc00d02..e59f324 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java
@@ -58,11 +58,13 @@ public abstract class FilterBase extends Filter {
   @Override
   @Deprecated
   public boolean filterRowKey(byte[] buffer, int offset, int length) throws IOException {
+    if (filterAllRemaining()) return true;
     return false;
   }
 
   @Override
   public boolean filterRowKey(Cell cell) throws IOException {
+    if (filterAllRemaining()) return true;
     return filterRowKey(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/2d203e60/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java
index 4d7a18a..617cd7a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java
@@ -103,11 +103,13 @@ final public class FilterWrapper extends Filter {
   @Override
   public boolean filterRowKey(byte[] buffer, int offset, int length) throws IOException {
     // No call to this.
+    if (filterAllRemaining()) return true;
     return this.filter.filterRowKey(buffer, offset, length);
   }
 
   @Override
   public boolean filterRowKey(Cell cell) throws IOException {
+    if (filterAllRemaining()) return true;
     return this.filter.filterRowKey(cell);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/2d203e60/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java
index 1096f5e..5dcb50d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java
@@ -61,6 +61,7 @@ public class InclusiveStopFilter extends FilterBase {
 
   public boolean filterRowKey(Cell firstRowCell) {
     // if stopRowKey is <= buffer, then true, filter row.
+    if (filterAllRemaining()) return true;
     int cmp = CellComparator.COMPARATOR.compareRows(firstRowCell, stopRowKey, 0, stopRowKey.length);
     done = reversed ? cmp < 0 : cmp > 0;
     return done;

http://git-wip-us.apache.org/repos/asf/hbase/blob/2d203e60/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultiRowRangeFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultiRowRangeFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultiRowRangeFilter.java
index 5f9c833..3f26586 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultiRowRangeFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultiRowRangeFilter.java
@@ -85,6 +85,7 @@ public class MultiRowRangeFilter extends FilterBase {
 
   @Override
   public boolean filterRowKey(Cell firstRowCell) {
+    if (filterAllRemaining()) return true;
     // If it is the first time of running, calculate the current range index for
     // the row key. If index is out of bound which happens when the start row
     // user sets is after the largest stop row of the ranges, stop the scan.

http://git-wip-us.apache.org/repos/asf/hbase/blob/2d203e60/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java
index adc9c54..f12fac8 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java
@@ -63,6 +63,7 @@ public class PageFilter extends FilterBase {
   @Override
   public boolean filterRowKey(Cell cell) throws IOException {
     // Impl in FilterBase might do unnecessary copy for Off heap backed Cells.
+    if (filterAllRemaining()) return true;
     return false;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/2d203e60/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java
index d09ea2c..e7b91e1 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java
@@ -55,6 +55,7 @@ public class PrefixFilter extends FilterBase {
   public boolean filterRowKey(Cell firstRowCell) {
     if (firstRowCell == null || this.prefix == null)
       return true;
+    if (filterAllRemaining()) return true;
     int length = firstRowCell.getRowLength();
     if (length < prefix.length) return true;
     // if they are equal, return false => pass row

http://git-wip-us.apache.org/repos/asf/hbase/blob/2d203e60/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java
index e75ca49..93b4a00 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java
@@ -74,6 +74,7 @@ public class WhileMatchFilter extends FilterBase {
 
   @Override
   public boolean filterRowKey(Cell cell) throws IOException {
+    if (filterAllRemaining()) return true;
     boolean value = filter.filterRowKey(cell);
     changeFAR(value);
     return value;


[47/50] [abbrv] hbase git commit: HBASE-16404 Make DeleteBloomFilters work with BloomContext (Ram)

Posted by sy...@apache.org.
HBASE-16404 Make DeleteBloomFilters work with BloomContext (Ram)


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

Branch: refs/heads/hbase-12439
Commit: 65c03b058c7b4a74f991acc07958be5e7d0e05e6
Parents: 285e104
Author: Ramkrishna <ra...@intel.com>
Authored: Fri Aug 12 15:09:31 2016 +0530
Committer: Ramkrishna <ra...@intel.com>
Committed: Fri Aug 12 15:09:31 2016 +0530

----------------------------------------------------------------------
 .../hbase/regionserver/StoreFileWriter.java     | 45 +++++++-------------
 .../apache/hadoop/hbase/util/BloomContext.java  |  9 ++--
 .../hadoop/hbase/util/RowBloomContext.java      |  4 +-
 3 files changed, 23 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/65c03b05/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java
index 47a8b25..cb5d12c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java
@@ -58,9 +58,9 @@ public class StoreFileWriter implements Compactor.CellSink {
   private final BloomFilterWriter deleteFamilyBloomFilterWriter;
   private final BloomType bloomType;
   private long earliestPutTs = HConstants.LATEST_TIMESTAMP;
-  private Cell lastDeleteFamilyCell = null;
   private long deleteFamilyCnt = 0;
   private BloomContext bloomContext = null;
+  private BloomContext deleteFamilyBloomContext = null;
 
   /**
    * timeRangeTrackerSet is used to figure if we were passed a filled-out TimeRangeTracker or not.
@@ -137,6 +137,18 @@ public class StoreFileWriter implements Compactor.CellSink {
         LOG.trace("Bloom filter type for " + path + ": " + this.bloomType + ", " +
             generalBloomFilterWriter.getClass().getSimpleName());
       }
+      // init bloom context
+      switch (bloomType) {
+      case ROW:
+        bloomContext = new RowBloomContext(generalBloomFilterWriter);
+        break;
+      case ROWCOL:
+        bloomContext = new RowColBloomContext(generalBloomFilterWriter);
+        break;
+      default:
+        throw new IOException(
+            "Invalid Bloom filter type: " + bloomType + " (ROW or ROWCOL expected)");
+      }
     } else {
       // Not using Bloom filters.
       this.bloomType = BloomType.NONE;
@@ -148,6 +160,7 @@ public class StoreFileWriter implements Compactor.CellSink {
       this.deleteFamilyBloomFilterWriter = BloomFilterFactory
           .createDeleteBloomAtWrite(conf, cacheConf,
               (int) Math.min(maxKeys, Integer.MAX_VALUE), writer);
+      deleteFamilyBloomContext = new RowBloomContext(deleteFamilyBloomFilterWriter);
     } else {
       deleteFamilyBloomFilterWriter = null;
     }
@@ -213,22 +226,6 @@ public class StoreFileWriter implements Compactor.CellSink {
 
   private void appendGeneralBloomfilter(final Cell cell) throws IOException {
     if (this.generalBloomFilterWriter != null) {
-      // only add to the bloom filter on a new, unique key
-      if (this.bloomContext == null) {
-        // init bloom context
-        switch (bloomType) {
-        case ROW:
-          bloomContext = new RowBloomContext(generalBloomFilterWriter);
-          break;
-        case ROWCOL:
-          bloomContext = new RowColBloomContext(generalBloomFilterWriter);
-          break;
-        default:
-          throw new IOException(
-              "Invalid Bloom filter type: " + bloomType + " (ROW or ROWCOL expected)");
-        }
-      }
-
       /*
        * http://2.bp.blogspot.com/_Cib_A77V54U/StZMrzaKufI/AAAAAAAAADo/ZhK7bGoJdMQ/s400/KeyValue.png
        * Key = RowLen + Row + FamilyLen + Column [Family + Qualifier] + TimeStamp
@@ -249,18 +246,8 @@ public class StoreFileWriter implements Compactor.CellSink {
 
     // increase the number of delete family in the store file
     deleteFamilyCnt++;
-    if (null != this.deleteFamilyBloomFilterWriter) {
-      boolean newKey = true;
-      if (lastDeleteFamilyCell != null) {
-        // hbase:meta does not have blooms. So we need not have special interpretation
-        // of the hbase:meta cells
-        newKey = !CellUtil.matchingRows(cell, lastDeleteFamilyCell);
-      }
-      // TODO : Use bloom context for delete family bloom filter also
-      if (newKey) {
-        this.deleteFamilyBloomFilterWriter.add(cell);
-        this.lastDeleteFamilyCell = cell;
-      }
+    if (this.deleteFamilyBloomFilterWriter != null) {
+      deleteFamilyBloomContext.writeBloom(cell);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/65c03b05/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomContext.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomContext.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomContext.java
index aab30e0..fc40aaf 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomContext.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomContext.java
@@ -33,10 +33,10 @@ public abstract class BloomContext {
   // TODO : Avoid holding references to lastCell
   protected Cell lastCell;
 
-  protected BloomFilterWriter generalBloomFilterWriter;
+  protected BloomFilterWriter bloomFilterWriter;
 
-  public BloomContext(BloomFilterWriter generalBloomFilterWriter) {
-    this.generalBloomFilterWriter = generalBloomFilterWriter;
+  public BloomContext(BloomFilterWriter bloomFilterWriter) {
+    this.bloomFilterWriter = bloomFilterWriter;
   }
 
   public Cell getLastCell() {
@@ -49,8 +49,9 @@ public abstract class BloomContext {
    * @throws IOException
    */
   public void writeBloom(Cell cell) throws IOException {
+    // only add to the bloom filter on a new, unique key
     if (isNewKey(cell)) {
-      generalBloomFilterWriter.add(cell);
+      bloomFilterWriter.add(cell);
       this.lastCell = cell;
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/65c03b05/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RowBloomContext.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RowBloomContext.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RowBloomContext.java
index c61e840..f6e36d4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RowBloomContext.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RowBloomContext.java
@@ -31,8 +31,8 @@ import org.apache.hadoop.hbase.regionserver.StoreFile;
 @InterfaceAudience.Private
 public class RowBloomContext extends BloomContext {
 
-  public RowBloomContext(BloomFilterWriter generalBloomFilterWriter) {
-    super(generalBloomFilterWriter);
+  public RowBloomContext(BloomFilterWriter bloomFilterWriter) {
+    super(bloomFilterWriter);
   }
 
   public void addLastBloomKey(Writer writer) throws IOException {


[38/50] [abbrv] hbase git commit: REVERT of revert of "HBASE-16308 Contain protobuf references Gather up the pb references into a few locations only rather than have pb references distributed all about the code base." This is a revert of a revert; i.e. w

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/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 882e21b..d2423b3 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
@@ -18,12 +18,6 @@
  */
 package org.apache.hadoop.hbase.client;
 
-import com.google.common.annotations.VisibleForTesting;
-import com.google.protobuf.Descriptors;
-import com.google.protobuf.Message;
-import com.google.protobuf.Service;
-import com.google.protobuf.ServiceException;
-
 import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.util.ArrayList;
@@ -43,7 +37,6 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.HTableDescriptor;
@@ -74,6 +67,15 @@ import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.ReflectionUtils;
 import org.apache.hadoop.hbase.util.Threads;
 
+import com.google.common.annotations.VisibleForTesting;
+// DO NOT MAKE USE OF THESE IMPORTS! THEY ARE HERE FOR COPROCESSOR ENDPOINTS ONLY.
+// Internally, we use shaded protobuf. This below are part of our public API.
+import com.google.protobuf.Descriptors;
+import com.google.protobuf.Message;
+import com.google.protobuf.Service;
+import com.google.protobuf.ServiceException;
+// SEE ABOVE NOTE!
+
 /**
  * An implementation of {@link Table}. Used to communicate with a single HBase table.
  * Lightweight. Get as needed and just close when done.
@@ -416,23 +418,16 @@ public class HTable implements Table {
 
     if (get.getConsistency() == Consistency.STRONG) {
       // Good old call.
-      final Get getReq = get;
+      final Get configuredGet = get;
       RegionServerCallable<Result> callable = new RegionServerCallable<Result>(this.connection,
-          getName(), get.getRow()) {
+          this.rpcControllerFactory, getName(), get.getRow()) {
         @Override
-        public Result call(int callTimeout) throws IOException {
-          ClientProtos.GetRequest request =
-            RequestConverter.buildGetRequest(getLocation().getRegionInfo().getRegionName(), getReq);
-          PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-          controller.setPriority(tableName);
-          controller.setCallTimeout(callTimeout);
-          try {
-            ClientProtos.GetResponse response = getStub().get(controller, request);
-            if (response == null) return null;
-            return ProtobufUtil.toResult(response.getResult(), controller.cellScanner());
-          } catch (ServiceException se) {
-            throw ProtobufUtil.getRemoteException(se);
-          }
+        protected Result rpcCall() throws Exception {
+          ClientProtos.GetRequest request = RequestConverter.buildGetRequest(
+              getLocation().getRegionInfo().getRegionName(), configuredGet);
+          ClientProtos.GetResponse response = getStub().get(getRpcController(), request);
+          if (response == null) return null;
+          return ProtobufUtil.toResult(response.getResult(), getRpcControllerCellScanner());
         }
       };
       return rpcCallerFactory.<Result>newCaller(readRpcTimeout).callWithRetries(callable,
@@ -448,7 +443,6 @@ public class HTable implements Table {
     return callable.call(operationTimeout);
   }
 
-
   /**
    * {@inheritDoc}
    */
@@ -459,16 +453,14 @@ public class HTable implements Table {
     }
     try {
       Object[] r1 = new Object[gets.size()];
-      batch((List) gets, r1);
-
-      // translate.
+      batch((List<? extends Row>)gets, r1);
+      // Translate.
       Result [] results = new Result[r1.length];
-      int i=0;
-      for (Object o : r1) {
-        // batch ensures if there is a failure we get an exception instead
-        results[i++] = (Result) o;
+      int i = 0;
+      for (Object obj: r1) {
+        // Batch ensures if there is a failure we get an exception instead
+        results[i++] = (Result)obj;
       }
-
       return results;
     } catch (InterruptedException e) {
       throw (InterruptedIOException)new InterruptedIOException().initCause(e);
@@ -516,21 +508,13 @@ public class HTable implements Table {
   public void delete(final Delete delete)
   throws IOException {
     RegionServerCallable<Boolean> callable = new RegionServerCallable<Boolean>(connection,
-        tableName, delete.getRow()) {
+        this.rpcControllerFactory, getName(), delete.getRow()) {
       @Override
-      public Boolean call(int callTimeout) throws IOException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setPriority(tableName);
-        controller.setCallTimeout(callTimeout);
-
-        try {
-          MutateRequest request = RequestConverter.buildMutateRequest(
-            getLocation().getRegionInfo().getRegionName(), delete);
-          MutateResponse response = getStub().mutate(controller, request);
-          return Boolean.valueOf(response.getProcessed());
-        } catch (ServiceException se) {
-          throw ProtobufUtil.getRemoteException(se);
-        }
+      protected Boolean rpcCall() throws Exception {
+        MutateRequest request = RequestConverter.buildMutateRequest(
+          getLocation().getRegionInfo().getRegionName(), delete);
+        MutateResponse response = getStub().mutate(getRpcController(), request);
+        return Boolean.valueOf(response.getProcessed());
       }
     };
     rpcCallerFactory.<Boolean> newCaller(writeRpcTimeout).callWithRetries(callable,
@@ -586,41 +570,28 @@ public class HTable implements Table {
    */
   @Override
   public void mutateRow(final RowMutations rm) throws IOException {
-    final RetryingTimeTracker tracker = new RetryingTimeTracker();
-    PayloadCarryingServerCallable<MultiResponse> callable =
-      new PayloadCarryingServerCallable<MultiResponse>(connection, getName(), rm.getRow(),
+    CancellableRegionServerCallable<MultiResponse> callable =
+      new CancellableRegionServerCallable<MultiResponse>(this.connection, getName(), rm.getRow(),
           rpcControllerFactory) {
-        @Override
-        public MultiResponse call(int callTimeout) throws IOException {
-          tracker.start();
-          controller.setPriority(tableName);
-          int remainingTime = tracker.getRemainingTime(callTimeout);
-          if (remainingTime == 0) {
-            throw new DoNotRetryIOException("Timeout for mutate row");
-          }
-          controller.setCallTimeout(remainingTime);
-          try {
-            RegionAction.Builder regionMutationBuilder = RequestConverter.buildRegionAction(
-                getLocation().getRegionInfo().getRegionName(), rm);
-            regionMutationBuilder.setAtomic(true);
-            MultiRequest request =
-                MultiRequest.newBuilder().addRegionAction(regionMutationBuilder.build()).build();
-            ClientProtos.MultiResponse response = getStub().multi(controller, request);
-            ClientProtos.RegionActionResult res = response.getRegionActionResultList().get(0);
-            if (res.hasException()) {
-              Throwable ex = ProtobufUtil.toException(res.getException());
-              if (ex instanceof IOException) {
-                throw (IOException) ex;
-              }
-              throw new IOException("Failed to mutate row: " +
-                  Bytes.toStringBinary(rm.getRow()), ex);
-            }
-            return ResponseConverter.getResults(request, response, controller.cellScanner());
-          } catch (ServiceException se) {
-            throw ProtobufUtil.getRemoteException(se);
+      @Override
+      protected MultiResponse rpcCall() throws Exception {
+        RegionAction.Builder regionMutationBuilder = RequestConverter.buildRegionAction(
+            getLocation().getRegionInfo().getRegionName(), rm);
+        regionMutationBuilder.setAtomic(true);
+        MultiRequest request =
+            MultiRequest.newBuilder().addRegionAction(regionMutationBuilder.build()).build();
+        ClientProtos.MultiResponse response = getStub().multi(getRpcController(), request);
+        ClientProtos.RegionActionResult res = response.getRegionActionResultList().get(0);
+        if (res.hasException()) {
+          Throwable ex = ProtobufUtil.toException(res.getException());
+          if (ex instanceof IOException) {
+            throw (IOException) ex;
           }
+          throw new IOException("Failed to mutate row: " + Bytes.toStringBinary(rm.getRow()), ex);
         }
-      };
+        return ResponseConverter.getResults(request, response, getRpcControllerCellScanner());
+      }
+    };
     AsyncRequestFuture ars = multiAp.submitAll(pool, tableName, rm.getMutations(),
         null, null, callable, operationTimeout);
     ars.waitUntilDone();
@@ -629,38 +600,32 @@ public class HTable implements Table {
     }
   }
 
+  private static void checkHasFamilies(final Mutation mutation) throws IOException {
+    if (mutation.numFamilies() == 0) {
+      throw new IOException("Invalid arguments to " + mutation + ", zero columns specified");
+    }
+  }
+
   /**
    * {@inheritDoc}
    */
   @Override
   public Result append(final Append append) throws IOException {
-    if (append.numFamilies() == 0) {
-      throw new IOException(
-          "Invalid arguments to append, no columns specified");
-    }
-
-    NonceGenerator ng = this.connection.getNonceGenerator();
-    final long nonceGroup = ng.getNonceGroup(), nonce = ng.newNonce();
-    RegionServerCallable<Result> callable =
-      new RegionServerCallable<Result>(this.connection, getName(), append.getRow()) {
-        @Override
-        public Result call(int callTimeout) throws IOException {
-          PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-          controller.setPriority(getTableName());
-          controller.setCallTimeout(callTimeout);
-          try {
-            MutateRequest request = RequestConverter.buildMutateRequest(
-              getLocation().getRegionInfo().getRegionName(), append, nonceGroup, nonce);
-            MutateResponse response = getStub().mutate(controller, request);
-            if (!response.hasResult()) return null;
-            return ProtobufUtil.toResult(response.getResult(), controller.cellScanner());
-          } catch (ServiceException se) {
-            throw ProtobufUtil.getRemoteException(se);
-          }
-        }
-      };
-    return rpcCallerFactory.<Result> newCaller(writeRpcTimeout).callWithRetries(callable,
-        this.operationTimeout);
+    checkHasFamilies(append);
+    NoncedRegionServerCallable<Result> callable =
+        new NoncedRegionServerCallable<Result>(this.connection,
+        this.rpcControllerFactory, getName(), append.getRow()) {
+      @Override
+      protected Result call(PayloadCarryingRpcController controller) throws Exception {
+        MutateRequest request = RequestConverter.buildMutateRequest(
+          getLocation().getRegionInfo().getRegionName(), append, getNonceGroup(), getNonce());
+        MutateResponse response = getStub().mutate(controller, request);
+        if (!response.hasResult()) return null;
+        return ProtobufUtil.toResult(response.getResult(), controller.cellScanner());
+      }
+    };
+    return rpcCallerFactory.<Result> newCaller(this.writeRpcTimeout).
+        callWithRetries(callable, this.operationTimeout);
   }
 
   /**
@@ -668,27 +633,17 @@ public class HTable implements Table {
    */
   @Override
   public Result increment(final Increment increment) throws IOException {
-    if (!increment.hasFamilies()) {
-      throw new IOException(
-          "Invalid arguments to increment, no columns specified");
-    }
-    NonceGenerator ng = this.connection.getNonceGenerator();
-    final long nonceGroup = ng.getNonceGroup(), nonce = ng.newNonce();
-    RegionServerCallable<Result> callable = new RegionServerCallable<Result>(this.connection,
-        getName(), increment.getRow()) {
+    checkHasFamilies(increment);
+    NoncedRegionServerCallable<Result> callable =
+        new NoncedRegionServerCallable<Result>(this.connection,
+        this.rpcControllerFactory, getName(), increment.getRow()) {
       @Override
-      public Result call(int callTimeout) throws IOException {
-        PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-        controller.setPriority(getTableName());
-        controller.setCallTimeout(callTimeout);
-        try {
-          MutateRequest request = RequestConverter.buildMutateRequest(
-            getLocation().getRegionInfo().getRegionName(), increment, nonceGroup, nonce);
-          MutateResponse response = getStub().mutate(controller, request);
-          return ProtobufUtil.toResult(response.getResult(), controller.cellScanner());
-        } catch (ServiceException se) {
-          throw ProtobufUtil.getRemoteException(se);
-        }
+      protected Result call(PayloadCarryingRpcController controller) throws Exception {
+        MutateRequest request = RequestConverter.buildMutateRequest(
+          getLocation().getRegionInfo().getRegionName(), increment, getNonceGroup(), getNonce());
+        MutateResponse response = getStub().mutate(controller, request);
+        // Should this check for null like append does?
+        return ProtobufUtil.toResult(response.getResult(), controller.cellScanner());
       }
     };
     return rpcCallerFactory.<Result> newCaller(writeRpcTimeout).callWithRetries(callable,
@@ -725,30 +680,21 @@ public class HTable implements Table {
           "Invalid arguments to incrementColumnValue", npe);
     }
 
-    NonceGenerator ng = this.connection.getNonceGenerator();
-    final long nonceGroup = ng.getNonceGroup(), nonce = ng.newNonce();
-    RegionServerCallable<Long> callable =
-      new RegionServerCallable<Long>(connection, getName(), row) {
-        @Override
-        public Long call(int callTimeout) throws IOException {
-          PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-          controller.setPriority(getTableName());
-          controller.setCallTimeout(callTimeout);
-          try {
-            MutateRequest request = RequestConverter.buildIncrementRequest(
-              getLocation().getRegionInfo().getRegionName(), row, family,
-              qualifier, amount, durability, nonceGroup, nonce);
-            MutateResponse response = getStub().mutate(controller, request);
-            Result result =
-              ProtobufUtil.toResult(response.getResult(), controller.cellScanner());
-            return Long.valueOf(Bytes.toLong(result.getValue(family, qualifier)));
-          } catch (ServiceException se) {
-            throw ProtobufUtil.getRemoteException(se);
-          }
-        }
-      };
-    return rpcCallerFactory.<Long> newCaller(writeRpcTimeout).callWithRetries(callable,
-        this.operationTimeout);
+    NoncedRegionServerCallable<Long> callable =
+        new NoncedRegionServerCallable<Long>(this.connection, this.rpcControllerFactory, getName(),
+            row) {
+      @Override
+      protected Long call(PayloadCarryingRpcController controller) throws Exception {
+        MutateRequest request = RequestConverter.buildIncrementRequest(
+          getLocation().getRegionInfo().getRegionName(), row, family,
+          qualifier, amount, durability, getNonceGroup(), getNonce());
+        MutateResponse response = getStub().mutate(controller, request);
+        Result result = ProtobufUtil.toResult(response.getResult(), controller.cellScanner());
+        return Long.valueOf(Bytes.toLong(result.getValue(family, qualifier)));
+      }
+    };
+    return rpcCallerFactory.<Long> newCaller(this.writeRpcTimeout).
+        callWithRetries(callable, this.operationTimeout);
   }
 
   /**
@@ -760,25 +706,19 @@ public class HTable implements Table {
       final Put put)
   throws IOException {
     RegionServerCallable<Boolean> callable =
-      new RegionServerCallable<Boolean>(connection, getName(), row) {
-        @Override
-        public Boolean call(int callTimeout) throws IOException {
-          PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-          controller.setPriority(tableName);
-          controller.setCallTimeout(callTimeout);
-          try {
-            MutateRequest request = RequestConverter.buildMutateRequest(
-                getLocation().getRegionInfo().getRegionName(), row, family, qualifier,
-                new BinaryComparator(value), CompareType.EQUAL, put);
-            MutateResponse response = getStub().mutate(controller, request);
-            return Boolean.valueOf(response.getProcessed());
-          } catch (ServiceException se) {
-            throw ProtobufUtil.getRemoteException(se);
-          }
-        }
-      };
-    return rpcCallerFactory.<Boolean> newCaller(writeRpcTimeout).callWithRetries(callable,
-        this.operationTimeout);
+        new RegionServerCallable<Boolean>(this.connection, this.rpcControllerFactory,
+            getName(), row) {
+      @Override
+      protected Boolean rpcCall() throws Exception {
+        MutateRequest request = RequestConverter.buildMutateRequest(
+          getLocation().getRegionInfo().getRegionName(), row, family, qualifier,
+          new BinaryComparator(value), CompareType.EQUAL, put);
+        MutateResponse response = getStub().mutate(getRpcController(), request);
+        return Boolean.valueOf(response.getProcessed());
+      }
+    };
+    return rpcCallerFactory.<Boolean> newCaller(this.writeRpcTimeout).
+        callWithRetries(callable, this.operationTimeout);
   }
 
   /**
@@ -790,56 +730,43 @@ public class HTable implements Table {
       final Put put)
   throws IOException {
     RegionServerCallable<Boolean> callable =
-      new RegionServerCallable<Boolean>(connection, getName(), row) {
-        @Override
-        public Boolean call(int callTimeout) throws IOException {
-          PayloadCarryingRpcController controller = new PayloadCarryingRpcController();
-          controller.setPriority(tableName);
-          controller.setCallTimeout(callTimeout);
-          try {
-            CompareType compareType = CompareType.valueOf(compareOp.name());
-            MutateRequest request = RequestConverter.buildMutateRequest(
-              getLocation().getRegionInfo().getRegionName(), row, family, qualifier,
-                new BinaryComparator(value), compareType, put);
-            MutateResponse response = getStub().mutate(controller, request);
-            return Boolean.valueOf(response.getProcessed());
-          } catch (ServiceException se) {
-            throw ProtobufUtil.getRemoteException(se);
-          }
-        }
-      };
-    return rpcCallerFactory.<Boolean> newCaller(writeRpcTimeout).callWithRetries(callable,
-        this.operationTimeout);
+        new RegionServerCallable<Boolean>(this.connection, this.rpcControllerFactory,
+            getName(), row) {
+      @Override
+      protected Boolean rpcCall() throws Exception {
+        CompareType compareType = CompareType.valueOf(compareOp.name());
+        MutateRequest request = RequestConverter.buildMutateRequest(
+          getLocation().getRegionInfo().getRegionName(), row, family, qualifier,
+          new BinaryComparator(value), compareType, put);
+        MutateResponse response = getStub().mutate(getRpcController(), request);
+        return Boolean.valueOf(response.getProcessed());
+      }
+    };
+    return rpcCallerFactory.<Boolean> newCaller(this.writeRpcTimeout).
+        callWithRetries(callable, this.operationTimeout);
   }
 
   /**
    * {@inheritDoc}
    */
   @Override
-  public boolean checkAndDelete(final byte [] row,
-      final byte [] family, final byte [] qualifier, final byte [] value,
-      final Delete delete)
+  public boolean checkAndDelete(final byte [] row, final byte [] family, final byte [] qualifier,
+      final byte [] value, final Delete delete)
   throws IOException {
     RegionServerCallable<Boolean> callable =
-      new RegionServerCallable<Boolean>(connection, getName(), row) {
-        @Override
-        public Boolean call(int callTimeout) throws IOException {
-          PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-          controller.setPriority(tableName);
-          controller.setCallTimeout(callTimeout);
-          try {
-            MutateRequest request = RequestConverter.buildMutateRequest(
-              getLocation().getRegionInfo().getRegionName(), row, family, qualifier,
-                new BinaryComparator(value), CompareType.EQUAL, delete);
-            MutateResponse response = getStub().mutate(controller, request);
-            return Boolean.valueOf(response.getProcessed());
-          } catch (ServiceException se) {
-            throw ProtobufUtil.getRemoteException(se);
-          }
-        }
-      };
-    return rpcCallerFactory.<Boolean> newCaller(writeRpcTimeout).callWithRetries(callable,
-        this.operationTimeout);
+        new RegionServerCallable<Boolean>(this.connection, this.rpcControllerFactory,
+            getName(), row) {
+      @Override
+      protected Boolean rpcCall() throws Exception {
+        MutateRequest request = RequestConverter.buildMutateRequest(
+          getLocation().getRegionInfo().getRegionName(), row, family, qualifier,
+          new BinaryComparator(value), CompareType.EQUAL, delete);
+        MutateResponse response = getStub().mutate(getRpcController(), request);
+        return Boolean.valueOf(response.getProcessed());
+      }
+    };
+    return rpcCallerFactory.<Boolean> newCaller(this.writeRpcTimeout).
+        callWithRetries(callable, this.operationTimeout);
   }
 
   /**
@@ -851,25 +778,19 @@ public class HTable implements Table {
       final Delete delete)
   throws IOException {
     RegionServerCallable<Boolean> callable =
-      new RegionServerCallable<Boolean>(connection, getName(), row) {
-        @Override
-        public Boolean call(int callTimeout) throws IOException {
-          PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-          controller.setPriority(tableName);
-          controller.setCallTimeout(callTimeout);
-          try {
-            CompareType compareType = CompareType.valueOf(compareOp.name());
-            MutateRequest request = RequestConverter.buildMutateRequest(
-              getLocation().getRegionInfo().getRegionName(), row, family, qualifier,
-                new BinaryComparator(value), compareType, delete);
-            MutateResponse response = getStub().mutate(controller, request);
-            return Boolean.valueOf(response.getProcessed());
-          } catch (ServiceException se) {
-            throw ProtobufUtil.getRemoteException(se);
-          }
-        }
-      };
-    return rpcCallerFactory.<Boolean> newCaller(writeRpcTimeout).callWithRetries(callable,
+        new RegionServerCallable<Boolean>(this.connection, this.rpcControllerFactory,
+            getName(), row) {
+      @Override
+      protected Boolean rpcCall() throws Exception {
+        CompareType compareType = CompareType.valueOf(compareOp.name());
+        MutateRequest request = RequestConverter.buildMutateRequest(
+          getLocation().getRegionInfo().getRegionName(), row, family, qualifier,
+          new BinaryComparator(value), compareType, delete);
+        MutateResponse response = getStub().mutate(getRpcController(), request);
+        return Boolean.valueOf(response.getProcessed());
+      }
+    };
+    return rpcCallerFactory.<Boolean> newCaller(this.writeRpcTimeout).callWithRetries(callable,
         this.operationTimeout);
   }
 
@@ -880,40 +801,29 @@ public class HTable implements Table {
   public boolean checkAndMutate(final byte [] row, final byte [] family, final byte [] qualifier,
     final CompareOp compareOp, final byte [] value, final RowMutations rm)
     throws IOException {
-    final RetryingTimeTracker tracker = new RetryingTimeTracker();
-    PayloadCarryingServerCallable<MultiResponse> callable =
-      new PayloadCarryingServerCallable<MultiResponse>(connection, getName(), rm.getRow(),
+    CancellableRegionServerCallable<MultiResponse> callable =
+      new CancellableRegionServerCallable<MultiResponse>(connection, getName(), rm.getRow(),
         rpcControllerFactory) {
         @Override
-        public MultiResponse call(int callTimeout) throws IOException {
-          tracker.start();
-          controller.setPriority(tableName);
-          int remainingTime = tracker.getRemainingTime(callTimeout);
-          if (remainingTime == 0) {
-            throw new DoNotRetryIOException("Timeout for mutate row");
-          }
-          controller.setCallTimeout(remainingTime);
-          try {
-            CompareType compareType = CompareType.valueOf(compareOp.name());
-            MultiRequest request = RequestConverter.buildMutateRequest(
-              getLocation().getRegionInfo().getRegionName(), row, family, qualifier,
-              new BinaryComparator(value), compareType, rm);
-            ClientProtos.MultiResponse response = getStub().multi(controller, request);
-            ClientProtos.RegionActionResult res = response.getRegionActionResultList().get(0);
-            if (res.hasException()) {
-              Throwable ex = ProtobufUtil.toException(res.getException());
-              if(ex instanceof IOException) {
-                throw (IOException)ex;
-              }
-              throw new IOException("Failed to checkAndMutate row: "+
-                                    Bytes.toStringBinary(rm.getRow()), ex);
+        protected MultiResponse rpcCall() throws Exception {
+          CompareType compareType = CompareType.valueOf(compareOp.name());
+          MultiRequest request = RequestConverter.buildMutateRequest(
+            getLocation().getRegionInfo().getRegionName(), row, family, qualifier,
+            new BinaryComparator(value), compareType, rm);
+          ClientProtos.MultiResponse response = getStub().multi(getRpcController(), request);
+          ClientProtos.RegionActionResult res = response.getRegionActionResultList().get(0);
+          if (res.hasException()) {
+            Throwable ex = ProtobufUtil.toException(res.getException());
+            if (ex instanceof IOException) {
+              throw (IOException)ex;
             }
-            return ResponseConverter.getResults(request, response, controller.cellScanner());
-          } catch (ServiceException se) {
-            throw ProtobufUtil.getRemoteException(se);
+            throw new IOException("Failed to checkAndMutate row: "+
+              Bytes.toStringBinary(rm.getRow()), ex);
           }
+          return ResponseConverter.getResults(request, response, getRpcControllerCellScanner());
         }
       };
+
     /**
      *  Currently, we use one array to store 'processed' flag which is returned by server.
      *  It is excessive to send such a large array, but that is required by the framework right now
@@ -973,7 +883,6 @@ public class HTable implements Table {
   }
 
   /**
-   * {@inheritDoc}
    * @throws IOException
    */
   void flushCommits() throws IOException {
@@ -1150,19 +1059,18 @@ public class HTable implements Table {
     for (final byte[] r : keys) {
       final RegionCoprocessorRpcChannel channel =
           new RegionCoprocessorRpcChannel(connection, tableName, r);
-      Future<R> future = pool.submit(
-          new Callable<R>() {
-            @Override
-            public R call() throws Exception {
-              T instance = ProtobufUtil.newServiceStub(service, channel);
-              R result = callable.call(instance);
-              byte[] region = channel.getLastRegion();
-              if (callback != null) {
-                callback.update(region, r, result);
-              }
-              return result;
-            }
-          });
+      Future<R> future = pool.submit(new Callable<R>() {
+        @Override
+        public R call() throws Exception {
+          T instance = ProtobufUtil.newServiceStub(service, channel);
+          R result = callable.call(instance);
+          byte[] region = channel.getLastRegion();
+          if (callback != null) {
+            callback.update(region, r, result);
+          }
+          return result;
+        }
+      });
       futures.put(r, future);
     }
     for (Map.Entry<byte[],Future<R>> e : futures.entrySet()) {
@@ -1236,9 +1144,6 @@ public class HTable implements Table {
     return tableName + ";" + connection;
   }
 
-  /**
-   * {@inheritDoc}
-   */
   @Override
   public <R extends Message> Map<byte[], R> batchCoprocessorService(
       Descriptors.MethodDescriptor methodDescriptor, Message request,
@@ -1247,14 +1152,13 @@ public class HTable implements Table {
         Bytes.BYTES_COMPARATOR));
     batchCoprocessorService(methodDescriptor, request, startKey, endKey, responsePrototype,
         new Callback<R>() {
-
-          @Override
-          public void update(byte[] region, byte[] row, R result) {
-            if (region != null) {
-              results.put(region, result);
-            }
-          }
-        });
+      @Override
+      public void update(byte[] region, byte[] row, R result) {
+        if (region != null) {
+          results.put(region, result);
+        }
+      }
+    });
     return results;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterCallable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterCallable.java
index 66d3c21..8c4da68 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterCallable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterCallable.java
@@ -21,16 +21,34 @@ package org.apache.hadoop.hbase.client;
 import java.io.Closeable;
 import java.io.IOException;
 
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
+import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.util.Bytes;
+
 /**
- * A RetryingCallable for master operations.
+ * A RetryingCallable for Master RPC operations.
+ * Implement the #rpcCall method. It will be retried on error. See its javadoc and the javadoc of
+ * #call(int). See {@link HBaseAdmin} for examples of how this is used. To get at the
+ * rpcController that has been created and configured to make this rpc call, use getRpcController().
+ * We are trying to contain all protobuf references including references to rpcController so we
+ * don't pollute codebase with protobuf references; keep the protobuf references contained and only
+ * present in a few classes rather than all about the code base.
+ * <p>Like {@link RegionServerCallable} only in here, we can safely be PayloadCarryingRpcController
+ * all the time. This is not possible in the similar {@link RegionServerCallable} Callable because
+ * it has to deal with Coprocessor Endpoints.
  * @param <V> return type
  */
 abstract class MasterCallable<V> implements RetryingCallable<V>, Closeable {
-  protected ClusterConnection connection;
+  protected final ClusterConnection connection;
   protected MasterKeepAliveConnection master;
+  private final PayloadCarryingRpcController rpcController;
 
-  public MasterCallable(final Connection connection) {
+  MasterCallable(final Connection connection, final RpcControllerFactory rpcConnectionFactory) {
     this.connection = (ClusterConnection) connection;
+    this.rpcController = rpcConnectionFactory.newController();
   }
 
   @Override
@@ -43,6 +61,7 @@ abstract class MasterCallable<V> implements RetryingCallable<V>, Closeable {
     // The above prepare could fail but this would still be called though masterAdmin is null
     if (this.master != null) {
       this.master.close();
+      this.master = null;
     }
   }
 
@@ -59,4 +78,65 @@ abstract class MasterCallable<V> implements RetryingCallable<V>, Closeable {
   public long sleep(long pause, int tries) {
     return ConnectionUtils.getPauseTime(pause, tries);
   }
+
+  /**
+   * Override that changes the {@link Callable#call()} Exception from {@link Exception} to
+   * {@link IOException}. It also does setup of an rpcController and calls through to the rpcCall()
+   * method which callers are expected to implement. If rpcController is an instance of
+   * PayloadCarryingRpcController, we will set a timeout on it.
+   */
+  @Override
+  // Same trick as in RegionServerCallable so users don't have to copy/paste so much boilerplate
+  // and so we contain references to protobuf. We can't set priority on the rpcController as
+  // we do in RegionServerCallable because we don't always have a Table when we call.
+  public V call(int callTimeout) throws IOException {
+    try {
+      if (this.rpcController != null) {
+        this.rpcController.setCallTimeout(callTimeout);
+      }
+      return rpcCall();
+    } catch (Exception e) {
+      throw ProtobufUtil.handleRemoteException(e);
+    }
+  }
+
+  /**
+   * Run the RPC call. Implement this method. To get at the rpcController that has been created
+   * and configured to make this rpc call, use getRpcController(). We are trying to contain
+   * rpcController references so we don't pollute codebase with protobuf references; keep the
+   * protobuf references contained and only present in a few classes rather than all about the
+   * code base.
+   * @throws Exception
+   */
+  protected abstract V rpcCall() throws Exception;
+
+  PayloadCarryingRpcController getRpcController() {
+    return this.rpcController;
+  }
+
+  void setPriority(final int priority) {
+    if (this.rpcController != null) {
+      this.rpcController.setPriority(priority);
+    }
+  }
+
+  void setPriority(final TableName tableName) {
+    if (this.rpcController != null) {
+      this.rpcController.setPriority(tableName);
+    }
+  }
+
+  /**
+   * @param regionName RegionName. If hbase:meta, we'll set high priority.
+   */
+  void setPriority(final byte [] regionName) {
+    if (isMetaRegion(regionName)) {
+      setPriority(TableName.META_TABLE_NAME);
+    }
+  }
+
+  private static boolean isMetaRegion(final byte[] regionName) {
+    return Bytes.equals(regionName, HRegionInfo.FIRST_META_REGIONINFO.getRegionName())
+        || Bytes.equals(regionName, HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes());
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterKeepAliveConnection.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterKeepAliveConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterKeepAliveConnection.java
index e445b78..47693f4 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterKeepAliveConnection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterKeepAliveConnection.java
@@ -33,8 +33,7 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos;
  * against the master on the MasterProtos.MasterService.BlockingInterface; but not by
  * final user code. Hence it's package protected.
  */
-interface MasterKeepAliveConnection
-extends MasterProtos.MasterService.BlockingInterface {
+interface MasterKeepAliveConnection extends MasterProtos.MasterService.BlockingInterface {
   // Do this instead of implement Closeable because closeable returning IOE is PITA.
   void close();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java
index e764ceb..1ce4aab 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java
@@ -30,8 +30,9 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
-import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.protobuf.ResponseConverter;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
@@ -41,15 +42,15 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.protobuf.ServiceException;
 
 /**
  * Callable that handles the <code>multi</code> method call going against a single
- * regionserver; i.e. A {@link RegionServerCallable} for the multi call (It is not a
- * {@link RegionServerCallable} that goes against multiple regions.
+ * regionserver; i.e. A RegionServerCallable for the multi call (It is NOT a
+ * RegionServerCallable that goes against multiple regions).
  * @param <R>
  */
-class MultiServerCallable<R> extends PayloadCarryingServerCallable<MultiResponse> {
+@InterfaceAudience.Private
+class MultiServerCallable<R> extends CancellableRegionServerCallable<MultiResponse> {
   private final MultiAction<R> multiAction;
   private final boolean cellBlock;
 
@@ -79,7 +80,7 @@ class MultiServerCallable<R> extends PayloadCarryingServerCallable<MultiResponse
   }
 
   @Override
-  public MultiResponse call(int callTimeout) throws IOException {
+  protected MultiResponse rpcCall() throws Exception {
     int countOfActions = this.multiAction.size();
     if (countOfActions <= 0) throw new DoNotRetryIOException("No Actions");
     MultiRequest.Builder multiRequestBuilder = MultiRequest.newBuilder();
@@ -98,10 +99,8 @@ class MultiServerCallable<R> extends PayloadCarryingServerCallable<MultiResponse
       regionActionBuilder.clear();
       regionActionBuilder.setRegion(RequestConverter.buildRegionSpecifier(
           HBaseProtos.RegionSpecifier.RegionSpecifierType.REGION_NAME, regionName));
-
-
       if (this.cellBlock) {
-        // Presize.  Presume at least a KV per Action.  There are likely more.
+        // Pre-size. Presume at least a KV per Action.  There are likely more.
         if (cells == null) cells = new ArrayList<CellScannable>(countOfActions);
         // Send data in cellblocks. The call to buildNoDataMultiRequest will skip RowMutations.
         // They have already been handled above. Guess at count of cells
@@ -114,20 +113,13 @@ class MultiServerCallable<R> extends PayloadCarryingServerCallable<MultiResponse
       multiRequestBuilder.addRegionAction(regionActionBuilder.build());
     }
 
-    // Controller optionally carries cell data over the proxy/service boundary and also
-    // optionally ferries cell response data back out again.
-    if (cells != null) controller.setCellScanner(CellUtil.createCellScanner(cells));
-    controller.setPriority(getTableName());
-    controller.setCallTimeout(callTimeout);
-    ClientProtos.MultiResponse responseProto;
-    ClientProtos.MultiRequest requestProto = multiRequestBuilder.build();
-    try {
-      responseProto = getStub().multi(controller, requestProto);
-    } catch (ServiceException e) {
-      throw ProtobufUtil.getRemoteException(e);
+    if (cells != null) {
+      setRpcControllerCellScanner(CellUtil.createCellScanner(cells));
     }
+    ClientProtos.MultiRequest requestProto = multiRequestBuilder.build();
+    ClientProtos.MultiResponse responseProto = getStub().multi(getRpcController(), requestProto);
     if (responseProto == null) return null; // Occurs on cancel
-    return ResponseConverter.getResults(requestProto, responseProto, controller.cellScanner());
+    return ResponseConverter.getResults(requestProto, responseProto, getRpcControllerCellScanner());
   }
 
   /**
@@ -151,4 +143,4 @@ class MultiServerCallable<R> extends PayloadCarryingServerCallable<MultiResponse
   ServerName getServerName() {
     return location.getServerName();
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-client/src/main/java/org/apache/hadoop/hbase/client/NoncedRegionServerCallable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/NoncedRegionServerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/NoncedRegionServerCallable.java
new file mode 100644
index 0000000..21e77bd
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/NoncedRegionServerCallable.java
@@ -0,0 +1,128 @@
+/**
+ * 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
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.client;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
+import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
+
+/**
+ * Implementations make an rpc call against a RegionService via a protobuf Service.
+ * Implement #rpcCall(RpcController) and then call {@link #call(int)} to
+ * trigger the rpc. The {@link #call(int)} eventually invokes your
+ * #rpcCall(RpcController) meanwhile saving you having to write a bunch of
+ * boilerplate. The {@link #call(int)} implementation is from {@link RpcRetryingCaller} so rpcs are
+ * retried on fail.
+ *
+ * <p>TODO: this class is actually tied to one region, because most of the paths make use of
+ *       the regioninfo part of location when building requests. The only reason it works for
+ *       multi-region requests (e.g. batch) is that they happen to not use the region parts.
+ *       This could be done cleaner (e.g. having a generic parameter and 2 derived classes,
+ *       RegionCallable and actual RegionServerCallable with ServerName.
+ * @param <T> the class that the ServerCallable handles
+ */
+@InterfaceAudience.Private
+public abstract class NoncedRegionServerCallable<T> extends AbstractRegionServerCallable<T> {
+  private ClientService.BlockingInterface stub;
+  private final PayloadCarryingRpcController rpcController;
+  private final long nonce;
+
+  /**
+   * @param connection Connection to use.
+   * @param tableName Table name to which <code>row</code> belongs.
+   * @param row The row we want in <code>tableName</code>.
+   */
+  public NoncedRegionServerCallable(Connection connection, RpcControllerFactory rpcControllerFactory,
+      TableName tableName, byte [] row) {
+    this(connection, rpcControllerFactory.newController(), tableName, row);
+  }
+
+  public NoncedRegionServerCallable(Connection connection, PayloadCarryingRpcController rpcController,
+      TableName tableName, byte [] row) {
+    super(connection, tableName, row);
+    this.rpcController = rpcController;
+    if (this.rpcController != null) {
+      this.rpcController.setPriority(tableName);
+    }
+    this.nonce = getConnection().getNonceGenerator().newNonce();
+  }
+
+  void setClientByServiceName(ServerName service) throws IOException {
+    this.setStub(getConnection().getClient(service));
+  }
+
+  /**
+   * @return Client Rpc protobuf communication stub
+   */
+  protected ClientService.BlockingInterface getStub() {
+    return this.stub;
+  }
+
+  /**
+   * Set the client protobuf communication stub
+   * @param stub to set
+   */
+  void setStub(final ClientService.BlockingInterface stub) {
+    this.stub = stub;
+  }
+
+  /**
+   * Override that changes Exception from {@link Exception} to {@link IOException}. It also does
+   * setup of an rpcController and calls through to the unimplemented
+   * call(PayloadCarryingRpcController) method; implement this method to add your rpc invocation.
+   */
+  @Override
+  public T call(int callTimeout) throws IOException {
+    if (this.rpcController != null) {
+      this.rpcController.setCallTimeout(callTimeout);
+    }
+    try {
+      return call(this.rpcController);
+    } catch (Exception e) {
+      throw ProtobufUtil.handleRemoteException(e);
+    }
+  }
+
+  /**
+   * Run RPC call.
+   * @param rpcController PayloadCarryingRpcController is a mouthful but it at a minimum is a
+   * facade on protobuf so we don't have to put protobuf everywhere; we can keep it behind this
+   * class.
+   * @throws Exception
+   */
+  protected abstract T call(PayloadCarryingRpcController rpcController) throws Exception;
+
+  public PayloadCarryingRpcController getRpcController() {
+    return this.rpcController;
+  }
+
+  long getNonceGroup() {
+    return getConnection().getNonceGenerator().getNonceGroup();
+  }
+
+  long getNonce() {
+    return this.nonce;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-client/src/main/java/org/apache/hadoop/hbase/client/PayloadCarryingServerCallable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/PayloadCarryingServerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/PayloadCarryingServerCallable.java
deleted file mode 100644
index d94f069..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/PayloadCarryingServerCallable.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * 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 regarding copyright ownership.
- * The ASF licenses this file to you under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.client;
-
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
-import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
-
-/**
- * This class is used to unify HTable calls with AsyncProcess Framework.
- * HTable can use AsyncProcess directly though this class.
- */
-@InterfaceAudience.Private
-public abstract class PayloadCarryingServerCallable<T>
-    extends RegionServerCallable<T> implements Cancellable {
-  protected PayloadCarryingRpcController controller;
-
-  public PayloadCarryingServerCallable(Connection connection, TableName tableName, byte[] row,
-    RpcControllerFactory rpcControllerFactory) {
-    super(connection, tableName, row);
-    this.controller = rpcControllerFactory.newController();
-  }
-
-  @Override
-  public void cancel() {
-    controller.startCancel();
-  }
-
-  @Override
-  public boolean isCancelled() {
-    return controller.isCanceled();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionAdminServiceCallable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionAdminServiceCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionAdminServiceCallable.java
index 54c93a0..4e347dd 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionAdminServiceCallable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionAdminServiceCallable.java
@@ -27,31 +27,30 @@ import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.RegionLocations;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
 import org.apache.hadoop.hbase.util.Bytes;
 
 /**
- * Similar to {@link RegionServerCallable} but for the AdminService interface. This service callable
+ * Similar to RegionServerCallable but for the AdminService interface. This service callable
  * assumes a Table and row and thus does region locating similar to RegionServerCallable.
+ * Works against Admin stub rather than Client stub.
  */
 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="URF_UNREAD_PUBLIC_OR_PROTECTED_FIELD",
   justification="stub used by ipc")
 @InterfaceAudience.Private
 public abstract class RegionAdminServiceCallable<T> implements RetryingCallable<T> {
-
-  protected final ClusterConnection connection;
-
-  protected final RpcControllerFactory rpcControllerFactory;
-
   protected AdminService.BlockingInterface stub;
+  protected final RpcControllerFactory rpcControllerFactory;
+  private PayloadCarryingRpcController controller = null;
 
+  protected final ClusterConnection connection;
   protected HRegionLocation location;
-
   protected final TableName tableName;
   protected final byte[] row;
   protected final int replicaId;
-
   protected final static int MIN_WAIT_DEAD_SERVER = 10000;
 
   public RegionAdminServiceCallable(ClusterConnection connection,
@@ -82,16 +81,13 @@ public abstract class RegionAdminServiceCallable<T> implements RetryingCallable<
     if (Thread.interrupted()) {
       throw new InterruptedIOException();
     }
-
     if (reload || location == null) {
       location = getLocation(!reload);
     }
-
     if (location == null) {
       // With this exception, there will be a retry.
       throw new HBaseIOException(getExceptionMessage());
     }
-
     this.setStub(connection.getAdmin(location.getServerName()));
   }
 
@@ -167,7 +163,39 @@ public abstract class RegionAdminServiceCallable<T> implements RetryingCallable<
     if (rl == null) {
       throw new RetriesExhaustedException("Can't get the locations");
     }
-
     return rl;
   }
-}
+
+  /**
+   * Override that changes Exception from {@link Exception} to {@link IOException}. It also does
+   * setup of an rpcController and calls through to the unimplemented
+   * call(PayloadCarryingRpcController) method; implement this method to add your rpc invocation.
+   */
+  @Override
+  // Same trick as in RegionServerCallable so users don't have to copy/paste so much boilerplate
+  // and so we contain references to protobuf. We can't set priority on the rpcController as
+  // we do in RegionServerCallable because we don't always have a Table when we call.
+  public T call(int callTimeout) throws IOException {
+    this.controller = rpcControllerFactory.newController();
+    this.controller.setPriority(this.tableName);
+    this.controller.setCallTimeout(callTimeout);
+    try {
+      return call(this.controller);
+    } catch (Exception e) {
+      throw ProtobufUtil.handleRemoteException(e);
+    }
+  }
+
+  PayloadCarryingRpcController getCurrentPayloadCarryingRpcController() {
+    return this.controller;
+  }
+
+  /**
+   * Run RPC call.
+   * @param rpcController PayloadCarryingRpcController is a mouthful but it at a minimum is a
+   * facade on protobuf so we don't have to put protobuf everywhere; we can keep it behind this
+   * class.
+   * @throws Exception
+   */
+  protected abstract T call(PayloadCarryingRpcController rpcController) throws Exception;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionServerCallable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionServerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionServerCallable.java
index d878bae..3771c50 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionServerCallable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionServerCallable.java
@@ -1,5 +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
@@ -21,34 +20,62 @@ package org.apache.hadoop.hbase.client;
 
 import java.io.IOException;
 
+import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
+import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
 
+import com.google.protobuf.RpcController;
+
 /**
- * Implementations call a RegionServer and implement {@link #call(int)}.
- * Passed to a {@link RpcRetryingCaller} so we retry on fail.
- * TODO: this class is actually tied to one region, because most of the paths make use of
+ * Implementations make an rpc call against a RegionService via a protobuf Service.
+ * Implement rpcCall(). Be sure to make use of the RpcController that this instance is carrying
+ * via {@link #getRpcController()}.
+ *
+ * <p>TODO: this class is actually tied to one region, because most of the paths make use of
  *       the regioninfo part of location when building requests. The only reason it works for
  *       multi-region requests (e.g. batch) is that they happen to not use the region parts.
  *       This could be done cleaner (e.g. having a generic parameter and 2 derived classes,
  *       RegionCallable and actual RegionServerCallable with ServerName.
+ *
  * @param <T> the class that the ServerCallable handles
  */
 @InterfaceAudience.Private
-public abstract class RegionServerCallable<T> extends AbstractRegionServerCallable<T> implements
-    RetryingCallable<T> {
-
+public abstract class RegionServerCallable<T> extends AbstractRegionServerCallable<T> {
   private ClientService.BlockingInterface stub;
 
+  /* This is 99% of the time a PayloadCarryingRpcController but this RegionServerCallable is
+   * also used doing Coprocessor Endpoints and in this case, it is a ServerRpcControllable which is
+   * not a PayloadCarryingRpcController. Too hard to untangle it all at this stage since
+   * downstreamers are using RegionServerCallable invoking CPEPs so just do ugly instanceof
+   * checks in the below.
+   */
+  private final RpcController rpcController;
+
   /**
    * @param connection Connection to use.
    * @param tableName Table name to which <code>row</code> belongs.
    * @param row The row we want in <code>tableName</code>.
    */
-  public RegionServerCallable(Connection connection, TableName tableName, byte [] row) {
+  public RegionServerCallable(Connection connection, RpcControllerFactory rpcControllerFactory,
+      TableName tableName, byte [] row) {
+    this(connection, rpcControllerFactory.newController(), tableName, row);
+  }
+
+  public RegionServerCallable(Connection connection, RpcController rpcController,
+      TableName tableName, byte [] row) {
     super(connection, tableName, row);
+    this.rpcController = rpcController;
+    // If it is an instance of PayloadCarryingRpcController, we can set priority on the
+    // controller based off the tableName. RpcController may be null in tests when mocking so allow
+    // for null controller.
+    if (this.rpcController != null && this.rpcController instanceof PayloadCarryingRpcController) {
+      ((PayloadCarryingRpcController)this.rpcController).setPriority(tableName);
+    }
   }
 
   void setClientByServiceName(ServerName service) throws IOException {
@@ -69,4 +96,55 @@ public abstract class RegionServerCallable<T> extends AbstractRegionServerCallab
   void setStub(final ClientService.BlockingInterface stub) {
     this.stub = stub;
   }
-}
+
+  /**
+   * Override that changes call Exception from {@link Exception} to {@link IOException}. It also
+   * does setup of an rpcController and calls through to the unimplemented
+   * rpcCall() method. If rpcController is an instance of PayloadCarryingRpcController,
+   * we will set a timeout on it.
+   */
+  @Override
+  public T call(int callTimeout) throws IOException {
+    try {
+      if (this.rpcController != null &&
+          this.rpcController instanceof PayloadCarryingRpcController) {
+        ((PayloadCarryingRpcController)this.rpcController).setCallTimeout(callTimeout);
+        // Do a reset of the CellScanner in case we are carrying any Cells since last time through.
+        setRpcControllerCellScanner(null);
+      }
+      return rpcCall();
+    } catch (Exception e) {
+      throw ProtobufUtil.handleRemoteException(e);
+    }
+  }
+
+  /**
+   * Run the RPC call. Implement this method. To get at the rpcController that has been created
+   * and configured to make this rpc call, use getRpcController(). We are trying to contain
+   * rpcController references so we don't pollute codebase with protobuf references; keep the
+   * protobuf references contained and only present in a few classes rather than all about the
+   * code base.
+   * @throws Exception
+   */
+  protected abstract T rpcCall() throws Exception;
+
+  protected RpcController getRpcController() {
+    return this.rpcController;
+  }
+
+  /**
+   * Get the RpcController CellScanner.
+   * If the RpcController is a PayloadCarryingRpcController, which it is in all cases except
+   * when we are processing Coprocessor Endpoint, then this method returns a reference to the
+   * CellScanner that the PayloadCarryingRpcController is carrying. Do it up here in this Callable
+   * so we don't have to scatter ugly instanceof tests around the codebase. Will fail if called in
+   * a Coproccessor Endpoint context. Should never happen.
+   */
+  protected CellScanner getRpcControllerCellScanner() {
+    return ((PayloadCarryingRpcController)this.rpcController).cellScanner();
+  }
+
+  protected void setRpcControllerCellScanner(CellScanner cellScanner) {
+    ((PayloadCarryingRpcController)this.rpcController).setCellScanner(cellScanner);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetryingCallable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetryingCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetryingCallable.java
index 2377a0d..afbcc9a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetryingCallable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetryingCallable.java
@@ -36,4 +36,4 @@ public interface RetryingCallable<T> extends RetryingCallableBase {
    * @throws Exception if unable to compute a result
    */
   T call(int callTimeout) throws Exception;
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetryingTimeTracker.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetryingTimeTracker.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetryingTimeTracker.java
index 24288e6..b9438e6 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetryingTimeTracker.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetryingTimeTracker.java
@@ -22,7 +22,6 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
  * Tracks the amount of time remaining for an operation.
  */
 class RetryingTimeTracker {
-
   private long globalStartTime = -1;
 
   public void start() {
@@ -38,16 +37,19 @@ class RetryingTimeTracker {
       if (callTimeout == Integer.MAX_VALUE) {
         return Integer.MAX_VALUE;
       }
-      int remainingTime = (int) (
-        callTimeout -
-        (EnvironmentEdgeManager.currentTime() - this.globalStartTime));
+      long remaining = EnvironmentEdgeManager.currentTime() - this.globalStartTime;
+      long remainingTime = callTimeout - remaining;
       if (remainingTime < 1) {
         // If there is no time left, we're trying anyway. It's too late.
         // 0 means no timeout, and it's not the intent here. So we secure both cases by
         // resetting to the minimum.
         remainingTime = 1;
       }
-      return remainingTime;
+      if (remainingTime > Integer.MAX_VALUE) {
+        throw new RuntimeException("remainingTime=" + remainingTime +
+            " which is > Integer.MAX_VALUE");
+      }
+      return (int)remainingTime;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedScannerCallable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedScannerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedScannerCallable.java
index 0c2d345..a5bebd0 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedScannerCallable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedScannerCallable.java
@@ -176,9 +176,9 @@ public class ReversedScannerCallable extends ScannerCallable {
 
   @Override
   public ScannerCallable getScannerCallableForReplica(int id) {
-    ReversedScannerCallable r = new ReversedScannerCallable(this.cConnection, this.tableName,
-        this.getScan(), this.scanMetrics, this.locateStartRow, controllerFactory, id);
+    ReversedScannerCallable r = new ReversedScannerCallable(getConnection(), this.tableName,
+        this.getScan(), this.scanMetrics, this.locateStartRow, rpcControllerFactory, id);
     r.setCaching(this.getCaching());
     return r;
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallable.java
new file mode 100644
index 0000000..68a4aa2
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallable.java
@@ -0,0 +1,65 @@
+/**
+ * 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
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.client;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+
+/**
+ * A RetryingCallable for RPC connection operations.
+ * @param <V> return type
+ */
+abstract class RpcRetryingCallable<V> implements RetryingCallable<V>, Closeable {
+  @Override
+  public void prepare(boolean reload) throws IOException {
+  }
+
+  @Override
+  public void close() throws IOException {
+  }
+
+  @Override
+  public void throwable(Throwable t, boolean retrying) {
+  }
+
+  @Override
+  public String getExceptionMessageAdditionalDetail() {
+    return "";
+  }
+
+  @Override
+  public long sleep(long pause, int tries) {
+    return ConnectionUtils.getPauseTime(pause, tries);
+  }
+
+  @Override
+  // Same trick as in RegionServerCallable so users don't have to copy/paste so much boilerplate
+  // and so we contain references to protobuf.
+  public V call(int callTimeout) throws IOException {
+    try {
+      return rpcCall(callTimeout);
+    } catch (Exception e) {
+      throw ProtobufUtil.handleRemoteException(e);
+    }
+  }
+
+  protected abstract V rpcCall(int callTimeout) throws Exception;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/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 b4cd2ef..2b2e4c8 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
@@ -22,9 +22,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 import java.io.IOException;
 
-/**
- *
- */
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
 public interface RpcRetryingCaller<T> {
@@ -52,4 +49,4 @@ public interface RpcRetryingCaller<T> {
    */
   T callWithoutRetries(RetryingCallable<T> callable, int callTimeout)
   throws IOException, RuntimeException;
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/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 1c723c5..f92aeae 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
@@ -36,6 +36,7 @@ public class RpcRetryingCallerFactory {
   private final int rpcTimeout;
   private final RetryingCallerInterceptor interceptor;
   private final int startLogErrorsCnt;
+  /* These below data members are UNUSED!!!*/
   private final boolean enableBackPressure;
   private ServerStatisticTracker stats;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerWithReadReplicas.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerWithReadReplicas.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerWithReadReplicas.java
index 65dbb10..8d63295 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerWithReadReplicas.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerWithReadReplicas.java
@@ -29,8 +29,6 @@ import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseIOException;
@@ -46,8 +44,6 @@ import org.apache.hadoop.hbase.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 
-import com.google.protobuf.ServiceException;
-
 
 /**
  * Caller that goes to replica if the primary region does no answer within a configurable
@@ -57,8 +53,6 @@ import com.google.protobuf.ServiceException;
  */
 @InterfaceAudience.Private
 public class RpcRetryingCallerWithReadReplicas {
-  private static final Log LOG = LogFactory.getLog(RpcRetryingCallerWithReadReplicas.class);
-
   protected final ExecutorService pool;
   protected final ClusterConnection cConnection;
   protected final Configuration conf;
@@ -98,7 +92,7 @@ public class RpcRetryingCallerWithReadReplicas {
     private final PayloadCarryingRpcController controller;
 
     public ReplicaRegionServerCallable(int id, HRegionLocation location) {
-      super(RpcRetryingCallerWithReadReplicas.this.cConnection,
+      super(RpcRetryingCallerWithReadReplicas.this.cConnection, rpcControllerFactory,
           RpcRetryingCallerWithReadReplicas.this.tableName, get.getRow());
       this.id = id;
       this.location = location;
@@ -141,28 +135,22 @@ public class RpcRetryingCallerWithReadReplicas {
     }
 
     @Override
-    public Result call(int callTimeout) throws Exception {
+    protected Result rpcCall() throws Exception {
       if (controller.isCanceled()) return null;
-
       if (Thread.interrupted()) {
         throw new InterruptedIOException();
       }
-
       byte[] reg = location.getRegionInfo().getRegionName();
-
       ClientProtos.GetRequest request =
           RequestConverter.buildGetRequest(reg, get);
-      controller.setCallTimeout(callTimeout);
-
-      try {
-        ClientProtos.GetResponse response = getStub().get(controller, request);
-        if (response == null) {
-          return null;
-        }
-        return ProtobufUtil.toResult(response.getResult(), controller.cellScanner());
-      } catch (ServiceException se) {
-        throw ProtobufUtil.getRemoteException(se);
+      // Presumption that we are passed a PayloadCarryingRpcController here!
+      PayloadCarryingRpcController pcrc = (PayloadCarryingRpcController)controller;
+      pcrc.setCallTimeout(callTimeout);
+      ClientProtos.GetResponse response = getStub().get(controller, request);
+      if (response == null) {
+        return null;
       }
+      return ProtobufUtil.toResult(response.getResult(), pcrc.cellScanner());
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java
index 72d69ec..0ee54d0 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java
@@ -28,7 +28,6 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseIOException;
@@ -52,9 +51,6 @@ import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.net.DNS;
 
-import com.google.protobuf.ServiceException;
-import com.google.protobuf.TextFormat;
-
 /**
  * Scanner operations such as create, next, etc.
  * Used by {@link ResultScanner}s made by {@link Table}. Passed to a retrying caller such as
@@ -74,7 +70,6 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
   protected boolean renew = false;
   private Scan scan;
   private int caching = 1;
-  protected final ClusterConnection cConnection;
   protected ScanMetrics scanMetrics;
   private boolean logScannerActivity = false;
   private int logCutOffLatency = 1000;
@@ -99,8 +94,7 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
   // indicate if it is a remote server call
   protected boolean isRegionServerRemote = true;
   private long nextCallSeq = 0;
-  protected RpcControllerFactory controllerFactory;
-  protected PayloadCarryingRpcController controller;
+  protected final RpcControllerFactory rpcControllerFactory;
 
   /**
    * @param connection which connection
@@ -125,19 +119,14 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
    */
   public ScannerCallable(ClusterConnection connection, TableName tableName, Scan scan,
       ScanMetrics scanMetrics, RpcControllerFactory rpcControllerFactory, int id) {
-    super(connection, tableName, scan.getStartRow());
+    super(connection, rpcControllerFactory, tableName, scan.getStartRow());
     this.id = id;
-    this.cConnection = connection;
     this.scan = scan;
     this.scanMetrics = scanMetrics;
     Configuration conf = connection.getConfiguration();
     logScannerActivity = conf.getBoolean(LOG_SCANNER_ACTIVITY, false);
     logCutOffLatency = conf.getInt(LOG_SCANNER_LATENCY_CUTOFF, 1000);
-    this.controllerFactory = rpcControllerFactory;
-  }
-
-  PayloadCarryingRpcController getController() {
-    return controller;
+    this.rpcControllerFactory = rpcControllerFactory;
   }
 
   /**
@@ -185,25 +174,16 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
     }
   }
 
-
-  @Override
-  public Result [] call(int callTimeout) throws IOException {
+  protected Result [] rpcCall() throws Exception {
     if (Thread.interrupted()) {
       throw new InterruptedIOException();
     }
-
-    if (controller == null) {
-      controller = controllerFactory.newController();
-      controller.setPriority(getTableName());
-      controller.setCallTimeout(callTimeout);
-    }
-
-    if (closed) {
-      if (scannerId != -1) {
+    if (this.closed) {
+      if (this.scannerId != -1) {
         close();
       }
     } else {
-      if (scannerId == -1L) {
+      if (this.scannerId == -1L) {
         this.scannerId = openScanner();
       } else {
         Result [] rrs = null;
@@ -212,61 +192,54 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
         setHeartbeatMessage(false);
         try {
           incRPCcallsMetrics();
-          request =
-              RequestConverter.buildScanRequest(scannerId, caching, false, nextCallSeq,
+          request = RequestConverter.buildScanRequest(scannerId, caching, false, nextCallSeq,
                 this.scanMetrics != null, renew);
           ScanResponse response = null;
-          try {
-            response = getStub().scan(controller, request);
-            // Client and RS maintain a nextCallSeq number during the scan. Every next() call
-            // from client to server will increment this number in both sides. Client passes this
-            // number along with the request and at RS side both the incoming nextCallSeq and its
-            // nextCallSeq will be matched. In case of a timeout this increment at the client side
-            // should not happen. If at the server side fetching of next batch of data was over,
-            // there will be mismatch in the nextCallSeq number. Server will throw
-            // OutOfOrderScannerNextException and then client will reopen the scanner with startrow
-            // as the last successfully retrieved row.
-            // See HBASE-5974
-            nextCallSeq++;
-            long timestamp = System.currentTimeMillis();
-            setHeartbeatMessage(response.hasHeartbeatMessage() && response.getHeartbeatMessage());
-            // Results are returned via controller
-            CellScanner cellScanner = controller.cellScanner();
-            rrs = ResponseConverter.getResults(cellScanner, response);
-            if (logScannerActivity) {
-              long now = System.currentTimeMillis();
-              if (now - timestamp > logCutOffLatency) {
-                int rows = rrs == null ? 0 : rrs.length;
-                LOG.info("Took " + (now-timestamp) + "ms to fetch "
+          response = getStub().scan(getRpcController(), request);
+          // Client and RS maintain a nextCallSeq number during the scan. Every next() call
+          // from client to server will increment this number in both sides. Client passes this
+          // number along with the request and at RS side both the incoming nextCallSeq and its
+          // nextCallSeq will be matched. In case of a timeout this increment at the client side
+          // should not happen. If at the server side fetching of next batch of data was over,
+          // there will be mismatch in the nextCallSeq number. Server will throw
+          // OutOfOrderScannerNextException and then client will reopen the scanner with startrow
+          // as the last successfully retrieved row.
+          // See HBASE-5974
+          nextCallSeq++;
+          long timestamp = System.currentTimeMillis();
+          setHeartbeatMessage(response.hasHeartbeatMessage() && response.getHeartbeatMessage());
+          rrs = ResponseConverter.getResults(getRpcControllerCellScanner(), response);
+          if (logScannerActivity) {
+            long now = System.currentTimeMillis();
+            if (now - timestamp > logCutOffLatency) {
+              int rows = rrs == null ? 0 : rrs.length;
+              LOG.info("Took " + (now-timestamp) + "ms to fetch "
                   + rows + " rows from scanner=" + scannerId);
-              }
             }
-            updateServerSideMetrics(response);
-            // moreResults is only used for the case where a filter exhausts all elements
-            if (response.hasMoreResults() && !response.getMoreResults()) {
-              scannerId = -1L;
-              closed = true;
-              // Implied that no results were returned back, either.
-              return null;
-            }
-            // moreResultsInRegion explicitly defines when a RS may choose to terminate a batch due
-            // to size or quantity of results in the response.
-            if (response.hasMoreResultsInRegion()) {
-              // Set what the RS said
-              setHasMoreResultsContext(true);
-              setServerHasMoreResults(response.getMoreResultsInRegion());
-            } else {
-              // Server didn't respond whether it has more results or not.
-              setHasMoreResultsContext(false);
-            }
-          } catch (ServiceException se) {
-            throw ProtobufUtil.getRemoteException(se);
+          }
+          updateServerSideMetrics(response);
+          // moreResults is only used for the case where a filter exhausts all elements
+          if (response.hasMoreResults() && !response.getMoreResults()) {
+            this.scannerId = -1L;
+            this.closed = true;
+            // Implied that no results were returned back, either.
+            return null;
+          }
+          // moreResultsInRegion explicitly defines when a RS may choose to terminate a batch due
+          // to size or quantity of results in the response.
+          if (response.hasMoreResultsInRegion()) {
+            // Set what the RS said
+            setHasMoreResultsContext(true);
+            setServerHasMoreResults(response.getMoreResultsInRegion());
+          } else {
+            // Server didn't respond whether it has more results or not.
+            setHasMoreResultsContext(false);
           }
           updateResultsMetrics(rrs);
         } catch (IOException e) {
           if (logScannerActivity) {
-            LOG.info("Got exception making request " + TextFormat.shortDebugString(request)
-              + " to " + getLocation(), e);
+            LOG.info("Got exception making request " + ProtobufUtil.toText(request) + " to " +
+                getLocation(), e);
           }
           IOException ioe = e;
           if (e instanceof RemoteException) {
@@ -275,9 +248,9 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
           if (logScannerActivity && (ioe instanceof UnknownScannerException)) {
             try {
               HRegionLocation location =
-                getConnection().relocateRegion(getTableName(), scan.getStartRow());
-              LOG.info("Scanner=" + scannerId
-                + " expired, current region location is " + location.toString());
+                  getConnection().relocateRegion(getTableName(), scan.getStartRow());
+              LOG.info("Scanner=" + scannerId + " expired, current region location is " +
+                  location.toString());
             } catch (Throwable t) {
               LOG.info("Failed to relocate region", t);
             }
@@ -375,9 +348,9 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
       ScanRequest request =
           RequestConverter.buildScanRequest(this.scannerId, 0, true, this.scanMetrics != null);
       try {
-        getStub().scan(controller, request);
-      } catch (ServiceException se) {
-        throw ProtobufUtil.getRemoteException(se);
+        getStub().scan(getRpcController(), request);
+      } catch (Exception e) {
+        throw ProtobufUtil.handleRemoteException(e);
       }
     } catch (IOException e) {
       LOG.warn("Ignore, probably already closed", e);
@@ -387,20 +360,18 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
 
   protected long openScanner() throws IOException {
     incRPCcallsMetrics();
-    ScanRequest request =
-      RequestConverter.buildScanRequest(
-        getLocation().getRegionInfo().getRegionName(),
-        this.scan, 0, false);
+    ScanRequest request = RequestConverter.buildScanRequest(
+        getLocation().getRegionInfo().getRegionName(), this.scan, 0, false);
     try {
-      ScanResponse response = getStub().scan(controller, request);
+      ScanResponse response = getStub().scan(getRpcController(), request);
       long id = response.getScannerId();
       if (logScannerActivity) {
         LOG.info("Open scanner=" + id + " for scan=" + scan.toString()
           + " on region " + getLocation().toString());
       }
       return id;
-    } catch (ServiceException se) {
-      throw ProtobufUtil.getRemoteException(se);
+    } catch (Exception e) {
+      throw ProtobufUtil.handleRemoteException(e);
     }
   }
 
@@ -443,11 +414,6 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
     return caching;
   }
 
-  @Override
-  public ClusterConnection getConnection() {
-    return cConnection;
-  }
-
   /**
    * Set the number of rows that will be fetched on next
    * @param caching the number of rows for caching
@@ -458,7 +424,7 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
 
   public ScannerCallable getScannerCallableForReplica(int id) {
     ScannerCallable s = new ScannerCallable(this.getConnection(), this.tableName,
-        this.getScan(), this.scanMetrics, controllerFactory, id);
+        this.getScan(), this.scanMetrics, this.rpcControllerFactory, id);
     s.setCaching(this.caching);
     return s;
   }
@@ -488,4 +454,4 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
   protected void setHasMoreResultsContext(boolean serverHasMoreResultsContext) {
     this.serverHasMoreResultsContext = serverHasMoreResultsContext;
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java
index c3a3834..096841b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java
@@ -267,7 +267,6 @@ class ScannerCallableWithReplicas implements RetryingCallable<Result[]> {
   /**
    * When a scanner switches in the middle of scanning (the 'next' call fails
    * for example), the upper layer {@link ClientScanner} needs to know
-   * @return
    */
   public boolean switchedToADifferentReplica() {
     return replicaSwitched.get();
@@ -398,8 +397,8 @@ class ScannerCallableWithReplicas implements RetryingCallable<Result[]> {
     public void cancel() {
       cancelled = true;
       caller.cancel();
-      if (callable.getController() != null) {
-        callable.getController().startCancel();
+      if (callable.getRpcController() != null) {
+        callable.getRpcController().startCancel();
       }
       someRPCcancelled = true;
     }


[45/50] [abbrv] hbase git commit: HBASE-16267 Remove commons-httpclient dependency from hbase-rest module

Posted by sy...@apache.org.
HBASE-16267 Remove commons-httpclient dependency from hbase-rest module


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

Branch: refs/heads/hbase-12439
Commit: fb668a0d3f8bde2e228527a9c45d0458ea7597e8
Parents: 9e116e0
Author: tedyu <yu...@gmail.com>
Authored: Wed Aug 10 21:28:45 2016 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Wed Aug 10 21:28:45 2016 -0700

----------------------------------------------------------------------
 hbase-rest/pom.xml                              |   4 +
 .../hadoop/hbase/rest/MultiRowResource.java     |   2 +-
 .../apache/hadoop/hbase/rest/client/Client.java | 210 ++++++++++++-------
 .../hadoop/hbase/rest/client/Response.java      |  21 +-
 .../hbase/rest/TestGetAndPutResource.java       |   4 +-
 .../hadoop/hbase/rest/TestGzipFilter.java       |  21 +-
 .../hadoop/hbase/rest/TestMultiRowResource.java |   5 +-
 .../hadoop/hbase/rest/TestScannerResource.java  |   2 +-
 .../hadoop/hbase/rest/TestSchemaResource.java   |   5 +-
 .../hbase/rest/client/TestRemoteTable.java      |  11 +-
 .../hadoop/hbase/http/log/TestLogLevel.java     |  16 +-
 pom.xml                                         |  10 +-
 12 files changed, 193 insertions(+), 118 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/fb668a0d/hbase-rest/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-rest/pom.xml b/hbase-rest/pom.xml
index 2e1b245..4d42a7a 100644
--- a/hbase-rest/pom.xml
+++ b/hbase-rest/pom.xml
@@ -279,6 +279,10 @@
       <artifactId>httpclient</artifactId>
     </dependency>
     <dependency>
+      <groupId>org.apache.httpcomponents</groupId>
+      <artifactId>httpcore</artifactId>
+    </dependency>
+    <dependency>
       <groupId>commons-lang</groupId>
       <artifactId>commons-lang</artifactId>
     </dependency>

http://git-wip-us.apache.org/repos/asf/hbase/blob/fb668a0d/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/MultiRowResource.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/MultiRowResource.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/MultiRowResource.java
index 9cafe27..b952c00 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/MultiRowResource.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/MultiRowResource.java
@@ -115,7 +115,7 @@ public class MultiRowResource extends ResourceBase implements Constants {
         servlet.getMetrics().incrementSucessfulGetRequests(1);
         return Response.ok(model).build();
       }
-    } catch (Exception e) {
+    } catch (IOException e) {
       servlet.getMetrics().incrementFailedGetRequests(1);
       return processException(e);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/fb668a0d/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/Client.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/Client.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/Client.java
index e26de63..204f688 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/Client.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/Client.java
@@ -19,30 +19,34 @@
 
 package org.apache.hadoop.hbase.rest.client;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
+import java.net.URI;
+import java.net.URISyntaxException;
 import java.util.Collections;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 
-import org.apache.commons.httpclient.Header;
-import org.apache.commons.httpclient.HttpClient;
-import org.apache.commons.httpclient.HttpMethod;
-import org.apache.commons.httpclient.HttpVersion;
-import org.apache.commons.httpclient.MultiThreadedHttpConnectionManager;
-import org.apache.commons.httpclient.URI;
-import org.apache.commons.httpclient.methods.ByteArrayRequestEntity;
-import org.apache.commons.httpclient.methods.DeleteMethod;
-import org.apache.commons.httpclient.methods.GetMethod;
-import org.apache.commons.httpclient.methods.HeadMethod;
-import org.apache.commons.httpclient.methods.PostMethod;
-import org.apache.commons.httpclient.methods.PutMethod;
-import org.apache.commons.httpclient.params.HttpClientParams;
-import org.apache.commons.httpclient.params.HttpConnectionManagerParams;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.http.Header;
+import org.apache.http.HttpResponse;
+import org.apache.http.client.HttpClient;
+import org.apache.http.client.methods.HttpDelete;
+import org.apache.http.client.methods.HttpGet;
+import org.apache.http.client.methods.HttpHead;
+import org.apache.http.client.methods.HttpPost;
+import org.apache.http.client.methods.HttpPut;
+import org.apache.http.client.methods.HttpUriRequest;
+import org.apache.http.entity.InputStreamEntity;
+import org.apache.http.impl.client.DefaultHttpClient;
+import org.apache.http.message.BasicHeader;
+import org.apache.http.params.CoreConnectionPNames;
+import org.apache.http.util.EntityUtils;
 
 /**
  * A wrapper around HttpClient which provides some useful function and
@@ -58,6 +62,8 @@ public class Client {
   private HttpClient httpClient;
   private Cluster cluster;
   private boolean sslEnabled;
+  private HttpResponse resp;
+  private HttpGet httpGet = null;
 
   private Map<String, String> extraHeaders;
 
@@ -71,18 +77,13 @@ public class Client {
   private void initialize(Cluster cluster, boolean sslEnabled) {
     this.cluster = cluster;
     this.sslEnabled = sslEnabled;
-    MultiThreadedHttpConnectionManager manager =
-      new MultiThreadedHttpConnectionManager();
-    HttpConnectionManagerParams managerParams = manager.getParams();
-    managerParams.setConnectionTimeout(2000); // 2 s
-    managerParams.setDefaultMaxConnectionsPerHost(10);
-    managerParams.setMaxTotalConnections(100);
     extraHeaders = new ConcurrentHashMap<String, String>();
-    this.httpClient = new HttpClient(manager);
-    HttpClientParams clientParams = httpClient.getParams();
-    clientParams.setVersion(HttpVersion.HTTP_1_1);
-
+    String clspath = System.getProperty("java.class.path");
+    LOG.debug("classpath " + clspath);
+    this.httpClient = new DefaultHttpClient();
+    this.httpClient.getParams().setIntParameter(CoreConnectionPNames.CONNECTION_TIMEOUT, 2000);
   }
+
   /**
    * Constructor
    * @param cluster the cluster definition
@@ -104,9 +105,6 @@ public class Client {
    * Shut down the client. Close any open persistent connections.
    */
   public void shutdown() {
-    MultiThreadedHttpConnectionManager manager =
-      (MultiThreadedHttpConnectionManager) httpClient.getHttpConnectionManager();
-    manager.shutdown();
   }
 
   /**
@@ -159,7 +157,7 @@ public class Client {
    * @return the HTTP response code
    * @throws IOException
    */
-  public int executePathOnly(Cluster cluster, HttpMethod method,
+  public HttpResponse executePathOnly(Cluster cluster, HttpUriRequest method,
       Header[] headers, String path) throws IOException {
     IOException lastException;
     if (cluster.nodes.size() < 1) {
@@ -178,10 +176,29 @@ public class Client {
         }
         sb.append(cluster.lastHost);
         sb.append(path);
-        URI uri = new URI(sb.toString(), true);
+        URI uri = new URI(sb.toString());
+        if (method instanceof HttpPut) {
+          HttpPut put = new HttpPut(uri);
+          put.setEntity(((HttpPut) method).getEntity());
+          put.setHeaders(method.getAllHeaders());
+          method = put;
+        } else if (method instanceof HttpGet) {
+          method = new HttpGet(uri);
+        } else if (method instanceof HttpHead) {
+          method = new HttpHead(uri);
+        } else if (method instanceof HttpDelete) {
+          method = new HttpDelete(uri);
+        } else if (method instanceof HttpPost) {
+          HttpPost post = new HttpPost(uri);
+          post.setEntity(((HttpPost) method).getEntity());
+          post.setHeaders(method.getAllHeaders());
+          method = post;
+        }
         return executeURI(method, headers, uri.toString());
       } catch (IOException e) {
         lastException = e;
+      } catch (URISyntaxException use) {
+        lastException = new IOException(use);
       }
     } while (++i != start && i < cluster.nodes.size());
     throw lastException;
@@ -195,25 +212,27 @@ public class Client {
    * @return the HTTP response code
    * @throws IOException
    */
-  public int executeURI(HttpMethod method, Header[] headers, String uri)
+  public HttpResponse executeURI(HttpUriRequest method, Header[] headers, String uri)
       throws IOException {
-    method.setURI(new URI(uri, true));
+    // method.setURI(new URI(uri, true));
     for (Map.Entry<String, String> e: extraHeaders.entrySet()) {
-      method.addRequestHeader(e.getKey(), e.getValue());
+      method.addHeader(e.getKey(), e.getValue());
     }
     if (headers != null) {
       for (Header header: headers) {
-        method.addRequestHeader(header);
+        method.addHeader(header);
       }
     }
     long startTime = System.currentTimeMillis();
-    int code = httpClient.executeMethod(method);
+    if (resp != null) EntityUtils.consumeQuietly(resp.getEntity());
+    resp = httpClient.execute(method);
+
     long endTime = System.currentTimeMillis();
     if (LOG.isTraceEnabled()) {
-      LOG.trace(method.getName() + " " + uri + " " + code + " " +
-        method.getStatusText() + " in " + (endTime - startTime) + " ms");
+      LOG.trace(method.getMethod() + " " + uri + " " + resp.getStatusLine().getStatusCode() + " " +
+          resp.getStatusLine().getReasonPhrase() + " in " + (endTime - startTime) + " ms");
     }
-    return code;
+    return resp;
   }
 
   /**
@@ -227,7 +246,7 @@ public class Client {
    * @return the HTTP response code
    * @throws IOException
    */
-  public int execute(Cluster cluster, HttpMethod method, Header[] headers,
+  public HttpResponse execute(Cluster cluster, HttpUriRequest method, Header[] headers,
       String path) throws IOException {
     if (path.startsWith("/")) {
       return executePathOnly(cluster, method, headers, path);
@@ -269,11 +288,10 @@ public class Client {
    */
   public Response head(Cluster cluster, String path, Header[] headers)
       throws IOException {
-    HeadMethod method = new HeadMethod();
+    HttpHead method = new HttpHead(path);
     try {
-      int code = execute(cluster, method, null, path);
-      headers = method.getResponseHeaders();
-      return new Response(code, headers, null);
+      HttpResponse resp = execute(cluster, method, null, path);
+      return new Response(resp.getStatusLine().getStatusCode(), resp.getAllHeaders(), null);
     } finally {
       method.releaseConnection();
     }
@@ -322,7 +340,7 @@ public class Client {
   public Response get(Cluster cluster, String path, String accept)
       throws IOException {
     Header[] headers = new Header[1];
-    headers[0] = new Header("Accept", accept);
+    headers[0] = new BasicHeader("Accept", accept);
     return get(cluster, path, headers);
   }
 
@@ -339,6 +357,45 @@ public class Client {
   }
 
   /**
+   * Returns the response body of the HTTPResponse, if any, as an array of bytes.
+   * If response body is not available or cannot be read, returns <tt>null</tt>
+   *
+   * Note: This will cause the entire response body to be buffered in memory. A
+   * malicious server may easily exhaust all the VM memory. It is strongly
+   * recommended, to use getResponseAsStream if the content length of the response
+   * is unknown or reasonably large.
+   *
+   * @param resp HttpResponse
+   * @return The response body, null if body is empty
+   * @throws IOException If an I/O (transport) problem occurs while obtaining the
+   * response body.
+   */
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value =
+      "NP_LOAD_OF_KNOWN_NULL_VALUE", justification = "null is possible return value")
+  public static byte[] getResponseBody(HttpResponse resp) throws IOException {
+    if (resp.getEntity() == null) return null;
+    try (InputStream instream = resp.getEntity().getContent()) {
+      if (instream != null) {
+        long contentLength = resp.getEntity().getContentLength();
+        if (contentLength > Integer.MAX_VALUE) {
+          //guard integer cast from overflow
+          throw new IOException("Content too large to be buffered: " + contentLength +" bytes");
+        }
+        ByteArrayOutputStream outstream = new ByteArrayOutputStream(
+            contentLength > 0 ? (int) contentLength : 4*1024);
+        byte[] buffer = new byte[4096];
+        int len;
+        while ((len = instream.read(buffer)) > 0) {
+          outstream.write(buffer, 0, len);
+        }
+        outstream.close();
+        return outstream.toByteArray();
+      }
+      return null;
+    }
+  }
+
+  /**
    * Send a GET request
    * @param c the cluster definition
    * @param path the path or URI
@@ -348,16 +405,13 @@ public class Client {
    */
   public Response get(Cluster c, String path, Header[] headers)
       throws IOException {
-    GetMethod method = new GetMethod();
-    try {
-      int code = execute(c, method, headers, path);
-      headers = method.getResponseHeaders();
-      byte[] body = method.getResponseBody();
-      InputStream in = method.getResponseBodyAsStream();
-      return new Response(code, headers, body, in);
-    } finally {
-      method.releaseConnection();
+    if (httpGet != null) {
+      httpGet.releaseConnection();
     }
+    httpGet = new HttpGet(path);
+    HttpResponse resp = execute(c, httpGet, headers, path);
+    return new Response(resp.getStatusLine().getStatusCode(), resp.getAllHeaders(),
+        resp, resp.getEntity() == null ? null : resp.getEntity().getContent());
   }
 
   /**
@@ -399,7 +453,7 @@ public class Client {
   public Response put(Cluster cluster, String path, String contentType,
       byte[] content) throws IOException {
     Header[] headers = new Header[1];
-    headers[0] = new Header("Content-Type", contentType);
+    headers[0] = new BasicHeader("Content-Type", contentType);
     return put(cluster, path, headers, content);
   }
 
@@ -417,7 +471,7 @@ public class Client {
       byte[] content, Header extraHdr) throws IOException {
     int cnt = extraHdr == null ? 1 : 2;
     Header[] headers = new Header[cnt];
-    headers[0] = new Header("Content-Type", contentType);
+    headers[0] = new BasicHeader("Content-Type", contentType);
     if (extraHdr != null) {
       headers[1] = extraHdr;
     }
@@ -450,13 +504,13 @@ public class Client {
    */
   public Response put(Cluster cluster, String path, Header[] headers,
       byte[] content) throws IOException {
-    PutMethod method = new PutMethod();
+    HttpPut method = new HttpPut(path);
     try {
-      method.setRequestEntity(new ByteArrayRequestEntity(content));
-      int code = execute(cluster, method, headers, path);
-      headers = method.getResponseHeaders();
-      content = method.getResponseBody();
-      return new Response(code, headers, content);
+      method.setEntity(new InputStreamEntity(new ByteArrayInputStream(content), content.length));
+      HttpResponse resp = execute(cluster, method, headers, path);
+      headers = resp.getAllHeaders();
+      content = getResponseBody(resp);
+      return new Response(resp.getStatusLine().getStatusCode(), headers, content);
     } finally {
       method.releaseConnection();
     }
@@ -501,7 +555,7 @@ public class Client {
   public Response post(Cluster cluster, String path, String contentType,
       byte[] content) throws IOException {
     Header[] headers = new Header[1];
-    headers[0] = new Header("Content-Type", contentType);
+    headers[0] = new BasicHeader("Content-Type", contentType);
     return post(cluster, path, headers, content);
   }
 
@@ -519,7 +573,7 @@ public class Client {
       byte[] content, Header extraHdr) throws IOException {
     int cnt = extraHdr == null ? 1 : 2;
     Header[] headers = new Header[cnt];
-    headers[0] = new Header("Content-Type", contentType);
+    headers[0] = new BasicHeader("Content-Type", contentType);
     if (extraHdr != null) {
       headers[1] = extraHdr;
     }
@@ -552,13 +606,13 @@ public class Client {
    */
   public Response post(Cluster cluster, String path, Header[] headers,
       byte[] content) throws IOException {
-    PostMethod method = new PostMethod();
+    HttpPost method = new HttpPost(path);
     try {
-      method.setRequestEntity(new ByteArrayRequestEntity(content));
-      int code = execute(cluster, method, headers, path);
-      headers = method.getResponseHeaders();
-      content = method.getResponseBody();
-      return new Response(code, headers, content);
+      method.setEntity(new InputStreamEntity(new ByteArrayInputStream(content), content.length));
+      HttpResponse resp = execute(cluster, method, headers, path);
+      headers = resp.getAllHeaders();
+      content = getResponseBody(resp);
+      return new Response(resp.getStatusLine().getStatusCode(), headers, content);
     } finally {
       method.releaseConnection();
     }
@@ -593,12 +647,12 @@ public class Client {
    * @throws IOException for error
    */
   public Response delete(Cluster cluster, String path) throws IOException {
-    DeleteMethod method = new DeleteMethod();
+    HttpDelete method = new HttpDelete(path);
     try {
-      int code = execute(cluster, method, null, path);
-      Header[] headers = method.getResponseHeaders();
-      byte[] content = method.getResponseBody();
-      return new Response(code, headers, content);
+      HttpResponse resp = execute(cluster, method, null, path);
+      Header[] headers = resp.getAllHeaders();
+      byte[] content = getResponseBody(resp);
+      return new Response(resp.getStatusLine().getStatusCode(), headers, content);
     } finally {
       method.releaseConnection();
     }
@@ -612,13 +666,13 @@ public class Client {
    * @throws IOException for error
    */
   public Response delete(Cluster cluster, String path, Header extraHdr) throws IOException {
-    DeleteMethod method = new DeleteMethod();
+    HttpDelete method = new HttpDelete(path);
     try {
       Header[] headers = { extraHdr };
-      int code = execute(cluster, method, headers, path);
-      headers = method.getResponseHeaders();
-      byte[] content = method.getResponseBody();
-      return new Response(code, headers, content);
+      HttpResponse resp = execute(cluster, method, headers, path);
+      headers = resp.getAllHeaders();
+      byte[] content = getResponseBody(resp);
+      return new Response(resp.getStatusLine().getStatusCode(), headers, content);
     } finally {
       method.releaseConnection();
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/fb668a0d/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/Response.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/Response.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/Response.java
index 871b646..27db365 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/Response.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/Response.java
@@ -19,11 +19,14 @@
 
 package org.apache.hadoop.hbase.rest.client;
 
+import java.io.IOException;
 import java.io.InputStream;
 
-import org.apache.commons.httpclient.Header;
+import org.apache.http.Header;
+import org.apache.http.HttpResponse;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.mortbay.log.Log;
 
 /**
  * The HTTP result code, response headers, and body of a HTTP response.
@@ -34,6 +37,7 @@ public class Response {
   private int code;
   private Header[] headers;
   private byte[] body;
+  private HttpResponse resp;
   private InputStream stream;
 
   /**
@@ -69,13 +73,15 @@ public class Response {
    * Constructor
    * @param code the HTTP response code
    * @param headers headers the HTTP response headers
-   * @param body the response body, can be null
+   * @param resp the response
    * @param in Inputstream if the response had one.
+   * Note: this is not thread-safe
    */
-  public Response(int code, Header[] headers, byte[] body, InputStream in) {
+  public Response(int code, Header[] headers, HttpResponse resp, InputStream in) {
     this.code = code;
     this.headers = headers;
-    this.body = body;
+    this.body = null;
+    this.resp = resp;
     this.stream = in;
   }
 
@@ -129,6 +135,13 @@ public class Response {
    * @return the HTTP response body
    */
   public byte[] getBody() {
+    if (body == null) {
+      try {
+        body = Client.getResponseBody(resp);
+      } catch (IOException ioe) {
+        Log.debug("encountered ioe when obtaining body", ioe);
+      }
+    }
     return body;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/fb668a0d/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGetAndPutResource.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGetAndPutResource.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGetAndPutResource.java
index c6fb2ff..d6eb1b3 100644
--- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGetAndPutResource.java
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGetAndPutResource.java
@@ -24,14 +24,12 @@ import java.io.ByteArrayInputStream;
 import java.io.IOException;
 import java.io.StringWriter;
 import java.net.URLEncoder;
-import java.util.Dictionary;
 import java.util.HashMap;
 import java.util.List;
-import java.util.Map;
 
 import javax.xml.bind.JAXBException;
 
-import org.apache.commons.httpclient.Header;
+import org.apache.http.Header;
 import org.apache.hadoop.hbase.CompatibilityFactory;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.rest.client.Response;

http://git-wip-us.apache.org/repos/asf/hbase/blob/fb668a0d/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGzipFilter.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGzipFilter.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGzipFilter.java
index 42d355d..5097454 100644
--- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGzipFilter.java
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGzipFilter.java
@@ -27,7 +27,8 @@ import java.io.ByteArrayOutputStream;
 import java.util.zip.GZIPInputStream;
 import java.util.zip.GZIPOutputStream;
 
-import org.apache.commons.httpclient.Header;
+import org.apache.http.Header;
+import org.apache.http.message.BasicHeader;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
@@ -95,8 +96,8 @@ public class TestGzipFilter {
     // input side filter
 
     Header[] headers = new Header[2];
-    headers[0] = new Header("Content-Type", Constants.MIMETYPE_BINARY);
-    headers[1] = new Header("Content-Encoding", "gzip");
+    headers[0] = new BasicHeader("Content-Type", Constants.MIMETYPE_BINARY);
+    headers[1] = new BasicHeader("Content-Encoding", "gzip");
     Response response = client.put(path, headers, value_1_gzip);
     assertEquals(response.getCode(), 200);
 
@@ -110,8 +111,8 @@ public class TestGzipFilter {
 
     // output side filter
 
-    headers[0] = new Header("Accept", Constants.MIMETYPE_BINARY);
-    headers[1] = new Header("Accept-Encoding", "gzip");
+    headers[0] = new BasicHeader("Accept", Constants.MIMETYPE_BINARY);
+    headers[1] = new BasicHeader("Accept-Encoding", "gzip");
     response = client.get(path, headers);
     assertEquals(response.getCode(), 200);
     ByteArrayInputStream bis = new ByteArrayInputStream(response.getBody());
@@ -128,8 +129,8 @@ public class TestGzipFilter {
   @Test
   public void testErrorNotGzipped() throws Exception {
     Header[] headers = new Header[2];
-    headers[0] = new Header("Accept", Constants.MIMETYPE_BINARY);
-    headers[1] = new Header("Accept-Encoding", "gzip");
+    headers[0] = new BasicHeader("Accept", Constants.MIMETYPE_BINARY);
+    headers[1] = new BasicHeader("Accept-Encoding", "gzip");
     Response response = client.get("/" + TABLE + "/" + ROW_1 + "/" + COLUMN_2, headers);
     assertEquals(response.getCode(), 404);
     String contentEncoding = response.getHeader("Content-Encoding");
@@ -142,9 +143,9 @@ public class TestGzipFilter {
 
   void testScannerResultCodes() throws Exception {
     Header[] headers = new Header[3];
-    headers[0] = new Header("Content-Type", Constants.MIMETYPE_XML);
-    headers[1] = new Header("Accept", Constants.MIMETYPE_JSON);
-    headers[2] = new Header("Accept-Encoding", "gzip");
+    headers[0] = new BasicHeader("Content-Type", Constants.MIMETYPE_XML);
+    headers[1] = new BasicHeader("Accept", Constants.MIMETYPE_JSON);
+    headers[2] = new BasicHeader("Accept-Encoding", "gzip");
     Response response = client.post("/" + TABLE + "/scanner", headers,
         "<Scanner/>".getBytes());
     assertEquals(response.getCode(), 201);

http://git-wip-us.apache.org/repos/asf/hbase/blob/fb668a0d/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestMultiRowResource.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestMultiRowResource.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestMultiRowResource.java
index 958cb15..84d1855 100644
--- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestMultiRowResource.java
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestMultiRowResource.java
@@ -18,7 +18,8 @@
  */
 package org.apache.hadoop.hbase.rest;
 
-import org.apache.commons.httpclient.Header;
+import org.apache.http.Header;
+import org.apache.http.message.BasicHeader;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.*;
 import org.apache.hadoop.hbase.client.Admin;
@@ -96,7 +97,7 @@ public class TestMultiRowResource {
   public static void setUpBeforeClass() throws Exception {
     conf = TEST_UTIL.getConfiguration();
     conf.setBoolean(RESTServer.REST_CSRF_ENABLED_KEY, csrfEnabled);
-    extraHdr = new Header(RESTServer.REST_CSRF_CUSTOM_HEADER_DEFAULT, "");
+    extraHdr = new BasicHeader(RESTServer.REST_CSRF_CUSTOM_HEADER_DEFAULT, "");
     TEST_UTIL.startMiniCluster();
     REST_TEST_UTIL.startServletContainer(conf);
     context = JAXBContext.newInstance(

http://git-wip-us.apache.org/repos/asf/hbase/blob/fb668a0d/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannerResource.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannerResource.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannerResource.java
index 5114b11..be7ee9a 100644
--- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannerResource.java
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannerResource.java
@@ -37,7 +37,7 @@ import javax.xml.bind.JAXBException;
 import javax.xml.bind.Marshaller;
 import javax.xml.bind.Unmarshaller;
 
-import org.apache.commons.httpclient.Header;
+import org.apache.http.Header;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;

http://git-wip-us.apache.org/repos/asf/hbase/blob/fb668a0d/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestSchemaResource.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestSchemaResource.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestSchemaResource.java
index d005445..19fdaf0 100644
--- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestSchemaResource.java
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestSchemaResource.java
@@ -28,7 +28,8 @@ import java.util.List;
 import javax.xml.bind.JAXBContext;
 import javax.xml.bind.JAXBException;
 
-import org.apache.commons.httpclient.Header;
+import org.apache.http.Header;
+import org.apache.http.message.BasicHeader;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
@@ -86,7 +87,7 @@ public class TestSchemaResource {
   public static void setUpBeforeClass() throws Exception {
     conf = TEST_UTIL.getConfiguration();
     conf.setBoolean(RESTServer.REST_CSRF_ENABLED_KEY, csrfEnabled);
-    extraHdr = new Header(RESTServer.REST_CSRF_CUSTOM_HEADER_DEFAULT, "");
+    extraHdr = new BasicHeader(RESTServer.REST_CSRF_CUSTOM_HEADER_DEFAULT, "");
     TEST_UTIL.startMiniCluster();
     REST_TEST_UTIL.startServletContainer(conf);
     client = new Client(new Cluster().add("localhost",

http://git-wip-us.apache.org/repos/asf/hbase/blob/fb668a0d/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java
index 19d0587..1ac37fa 100644
--- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java
@@ -30,7 +30,8 @@ import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
 
-import org.apache.commons.httpclient.Header;
+import org.apache.http.Header;
+import org.apache.http.message.BasicHeader;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
@@ -513,16 +514,16 @@ public class TestRemoteTable {
     Response response = new Response(200);
     assertEquals(200, response.getCode());
     Header[] headers = new Header[2];
-    headers[0] = new Header("header1", "value1");
-    headers[1] = new Header("header2", "value2");
+    headers[0] = new BasicHeader("header1", "value1");
+    headers[1] = new BasicHeader("header2", "value2");
     response = new Response(200, headers);
     assertEquals("value1", response.getHeader("header1"));
     assertFalse(response.hasBody());
     response.setCode(404);
     assertEquals(404, response.getCode());
     headers = new Header[2];
-    headers[0] = new Header("header1", "value1.1");
-    headers[1] = new Header("header2", "value2");
+    headers[0] = new BasicHeader("header1", "value1.1");
+    headers[1] = new BasicHeader("header2", "value2");
     response.setHeaders(headers);
     assertEquals("value1.1", response.getHeader("header1"));
     response.setBody(Bytes.toBytes("body"));

http://git-wip-us.apache.org/repos/asf/hbase/blob/fb668a0d/hbase-server/src/test/java/org/apache/hadoop/hbase/http/log/TestLogLevel.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/http/log/TestLogLevel.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/log/TestLogLevel.java
index d7942d1..e14e3b4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/http/log/TestLogLevel.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/log/TestLogLevel.java
@@ -61,18 +61,12 @@ public class TestLogLevel {
             .getConnectorAddress(0));
 
         //servlet
-        URL url = new URL("http://" + authority + "/logLevel?log=" + logName
-            + "&level=" + Level.ERROR);
+        URL url =
+            new URL("http://" + authority + "/logLevel?log=" + logName + "&level=" + Level.ERROR);
         out.println("*** Connecting to " + url);
-        HttpURLConnection connection = (HttpURLConnection)url.openConnection();
-        connection.connect();
-
-        BufferedReader in = new BufferedReader(new InputStreamReader(
-            connection.getInputStream()));
-        for(String line; (line = in.readLine()) != null; out.println(line));
-        in.close();
-        connection.disconnect();
-
+        try (BufferedReader in = new BufferedReader(new InputStreamReader(url.openStream()))) {
+          for(String line; (line = in.readLine()) != null; out.println(line));
+        }
         log.debug("log.debug2");
         log.info("log.info2");
         log.error("log.error2");

http://git-wip-us.apache.org/repos/asf/hbase/blob/fb668a0d/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 6051ba5..e1d7181 100644
--- a/pom.xml
+++ b/pom.xml
@@ -1275,7 +1275,7 @@
     <disruptor.version>3.3.0</disruptor.version>
     <!-- Do not use versions earlier than 3.2.2 due to a security vulnerability -->
     <collections.version>3.2.2</collections.version>
-    <httpclient.version>4.3.6</httpclient.version>
+    <httpclient.version>4.5.2</httpclient.version>
     <httpcore.version>4.4.4</httpcore.version>
     <metrics-core.version>3.1.2</metrics-core.version>
     <guava.version>12.0.1</guava.version>
@@ -2262,6 +2262,10 @@
             <artifactId>hadoop-minicluster</artifactId>
             <version>${hadoop-two.version}</version>
             <exclusions>
+            <exclusion>
+              <groupId>commons-httpclient</groupId>
+              <artifactId>commons-httpclient</artifactId>
+            </exclusion>
               <exclusion>
                 <groupId>javax.servlet.jsp</groupId>
                 <artifactId>jsp-api</artifactId>
@@ -2449,6 +2453,10 @@
            <artifactId>hadoop-minicluster</artifactId>
            <version>${hadoop-three.version}</version>
            <exclusions>
+            <exclusion>
+              <groupId>commons-httpclient</groupId>
+              <artifactId>commons-httpclient</artifactId>
+            </exclusion>
              <exclusion>
                <groupId>javax.servlet.jsp</groupId>
                <artifactId>jsp-api</artifactId>


[06/50] [abbrv] hbase git commit: HBASE-16359 NullPointerException in RSRpcServices.openRegion()

Posted by sy...@apache.org.
HBASE-16359 NullPointerException in RSRpcServices.openRegion()


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

Branch: refs/heads/hbase-12439
Commit: 5e23b3aad5fffbb564d793f1d722d152e39f628a
Parents: c9f84e8
Author: tedyu <yu...@gmail.com>
Authored: Thu Aug 4 20:46:48 2016 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Thu Aug 4 20:46:48 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java  | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/5e23b3aa/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index f9b78e1..0de4a28 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -1741,6 +1741,9 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
           } else {
             regionServer.updateRegionFavoredNodesMapping(region.getEncodedName(),
               regionOpenInfo.getFavoredNodesList());
+            if (htd == null) {
+              throw new IOException("Missing table descriptor for " + region.getEncodedName());
+            }
             if (htd.getPriority() >= HConstants.ADMIN_QOS || region.getTable().isSystemTable()) {
               regionServer.service.submit(new OpenPriorityRegionHandler(
                 regionServer, regionServer, region, htd, masterSystemTime));


[10/50] [abbrv] hbase git commit: HBASE-16321 ensure no findbugs-jsr305

Posted by sy...@apache.org.
HBASE-16321 ensure no findbugs-jsr305

Signed-off-by: Esteban Gutierrez <es...@apache.org>


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

Branch: refs/heads/hbase-12439
Commit: 60b79e2daa41927f5a2e00b78cca855f05048f0a
Parents: 4b3e45f
Author: Sean Busbey <bu...@apache.org>
Authored: Tue Aug 2 01:17:59 2016 -0500
Committer: Sean Busbey <bu...@apache.org>
Committed: Fri Aug 5 10:16:29 2016 -0500

----------------------------------------------------------------------
 hbase-client/pom.xml                            |  8 +-
 .../apache/hadoop/hbase/MetaTableAccessor.java  | 10 +--
 .../hbase/client/ConnectionImplementation.java  |  2 +-
 .../hbase/protobuf/ResponseConverter.java       |  2 +-
 .../apache/hadoop/hbase/codec/BaseDecoder.java  |  5 +-
 hbase-rsgroup/pom.xml                           | 12 +--
 hbase-server/pom.xml                            |  4 +-
 .../hbase/coprocessor/ObserverContext.java      |  3 +-
 .../hadoop/hbase/master/LoadBalancer.java       |  2 +-
 .../hadoop/hbase/master/TableStateManager.java  |  6 +-
 .../hadoop/hbase/util/FSTableDescriptors.java   |  2 +-
 .../hadoop/hbase/HBaseTestingUtility.java       |  3 +-
 .../hbase/client/TestMetaWithReplicas.java      |  5 +-
 .../TestMultiTableSnapshotInputFormat.java      |  2 +-
 hbase-spark/pom.xml                             | 39 +++++++++
 pom.xml                                         | 87 ++++++++++++++------
 src/main/asciidoc/_chapters/developer.adoc      |  4 +-
 17 files changed, 133 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/60b79e2d/hbase-client/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-client/pom.xml b/hbase-client/pom.xml
index 7fc4672..b78f198 100644
--- a/hbase-client/pom.xml
+++ b/hbase-client/pom.xml
@@ -247,8 +247,8 @@
       </activation>
       <dependencies>
         <dependency>
-           <groupId>com.google.code.findbugs</groupId>
-           <artifactId>jsr305</artifactId>
+           <groupId>com.github.stephenc.findbugs</groupId>
+           <artifactId>findbugs-annotations</artifactId>
            <optional>true</optional>
         </dependency>
         <dependency>
@@ -260,10 +260,6 @@
           <artifactId>hadoop-common</artifactId>
           <exclusions>
             <exclusion>
-              <groupId>com.github.stephenc.findbugs</groupId>
-              <artifactId>findbugs-annotations</artifactId>
-            </exclusion>
-            <exclusion>
               <groupId>net.java.dev.jets3t</groupId>
               <artifactId>jets3t</artifactId>
             </exclusion>

http://git-wip-us.apache.org/repos/asf/hbase/blob/60b79e2d/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
index 3d40c70..a5dbc94 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
@@ -17,8 +17,6 @@
  */
 package org.apache.hadoop.hbase;
 
-import javax.annotation.Nonnull;
-import javax.annotation.Nullable;
 import java.io.Closeable;
 import java.io.IOException;
 import java.io.InterruptedIOException;
@@ -34,6 +32,8 @@ import java.util.TreeMap;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
+import edu.umd.cs.findbugs.annotations.NonNull;
+import edu.umd.cs.findbugs.annotations.Nullable;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.protobuf.ServiceException;
 import org.apache.commons.logging.Log;
@@ -795,10 +795,10 @@ public class MetaTableAccessor {
    * @return Get closest metatable region row to passed <code>row</code>
    * @throws java.io.IOException
    */
-  @Nonnull
+  @NonNull
   public static HRegionInfo getClosestRegionInfo(Connection connection,
-      @Nonnull final TableName tableName,
-      @Nonnull final byte[] row)
+      @NonNull final TableName tableName,
+      @NonNull final byte[] row)
       throws IOException {
     byte[] searchRow = HRegionInfo.createRegionName(tableName, row, HConstants.NINES, false);
     Scan scan = getMetaScan(connection, 1);

http://git-wip-us.apache.org/repos/asf/hbase/blob/60b79e2d/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
index bb5c996..8dcda13 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
@@ -43,7 +43,7 @@ import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 
-import javax.annotation.Nullable;
+import edu.umd.cs.findbugs.annotations.Nullable;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;

http://git-wip-us.apache.org/repos/asf/hbase/blob/60b79e2d/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ResponseConverter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ResponseConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ResponseConverter.java
index 421907d..76b4ccf 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ResponseConverter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ResponseConverter.java
@@ -17,13 +17,13 @@
  */
 package org.apache.hadoop.hbase.protobuf;
 
-import javax.annotation.Nullable;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import edu.umd.cs.findbugs.annotations.Nullable;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.Cell;

http://git-wip-us.apache.org/repos/asf/hbase/blob/60b79e2d/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/BaseDecoder.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/BaseDecoder.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/BaseDecoder.java
index 86f8678..485b88a 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/BaseDecoder.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/BaseDecoder.java
@@ -22,8 +22,7 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.PushbackInputStream;
 
-import javax.annotation.Nonnull;
-
+import edu.umd.cs.findbugs.annotations.NonNull;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.Cell;
@@ -99,7 +98,7 @@ public abstract class BaseDecoder implements Codec.Decoder {
    * thrown if EOF is reached prematurely. Does not return null.
    * @throws IOException
    */
-  @Nonnull
+  @NonNull
   protected abstract Cell parseCell() throws IOException;
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/60b79e2d/hbase-rsgroup/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/pom.xml b/hbase-rsgroup/pom.xml
index aff49b7..d216263 100644
--- a/hbase-rsgroup/pom.xml
+++ b/hbase-rsgroup/pom.xml
@@ -259,8 +259,8 @@
       </activation>
       <dependencies>
         <dependency>
-           <groupId>com.google.code.findbugs</groupId>
-           <artifactId>jsr305</artifactId>
+           <groupId>com.github.stephenc.findbugs</groupId>
+           <artifactId>findbugs-annotations</artifactId>
            <optional>true</optional>
         </dependency>
         <dependency>
@@ -272,10 +272,6 @@
           <artifactId>hadoop-common</artifactId>
           <exclusions>
             <exclusion>
-              <groupId>com.github.stephenc.findbugs</groupId>
-              <artifactId>findbugs-annotations</artifactId>
-            </exclusion>
-            <exclusion>
               <groupId>net.java.dev.jets3t</groupId>
               <artifactId>jets3t</artifactId>
             </exclusion>
@@ -311,6 +307,10 @@
               <groupId>tomcat</groupId>
               <artifactId>jasper-runtime</artifactId>
             </exclusion>
+            <exclusion>
+              <groupId>com.google.code.findbugs</groupId>
+              <artifactId>jsr305</artifactId>
+            </exclusion>
           </exclusions>
         </dependency>
       </dependencies>

http://git-wip-us.apache.org/repos/asf/hbase/blob/60b79e2d/hbase-server/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-server/pom.xml b/hbase-server/pom.xml
index ff001b7..6cf1bb6 100644
--- a/hbase-server/pom.xml
+++ b/hbase-server/pom.xml
@@ -430,8 +430,8 @@
     </dependency>
     <!-- General dependencies -->
     <dependency>
-       <groupId>com.google.code.findbugs</groupId>
-       <artifactId>jsr305</artifactId>
+       <groupId>com.github.stephenc.findbugs</groupId>
+       <artifactId>findbugs-annotations</artifactId>
        <optional>true</optional>
     </dependency>
     <dependency>

http://git-wip-us.apache.org/repos/asf/hbase/blob/60b79e2d/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/ObserverContext.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/ObserverContext.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/ObserverContext.java
index d522ce9..52f2b95 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/ObserverContext.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/ObserverContext.java
@@ -19,6 +19,7 @@
 
 package org.apache.hadoop.hbase.coprocessor;
 
+import edu.umd.cs.findbugs.annotations.Nullable;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
@@ -26,8 +27,6 @@ import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.ipc.RpcServer;
 import org.apache.hadoop.hbase.security.User;
 
-import javax.annotation.Nullable;
-
 /**
  * Carries the execution state for a given invocation of an Observer coprocessor
  * ({@link RegionObserver}, {@link MasterObserver}, or {@link WALObserver})

http://git-wip-us.apache.org/repos/asf/hbase/blob/60b79e2d/hbase-server/src/main/java/org/apache/hadoop/hbase/master/LoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/LoadBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/LoadBalancer.java
index 5d6be9e..d7111c3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/LoadBalancer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/LoadBalancer.java
@@ -18,10 +18,10 @@
  */
 package org.apache.hadoop.hbase.master;
 
-import javax.annotation.Nullable;
 import java.util.List;
 import java.util.Map;
 
+import edu.umd.cs.findbugs.annotations.Nullable;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.conf.ConfigurationObserver;
 import org.apache.hadoop.conf.Configurable;

http://git-wip-us.apache.org/repos/asf/hbase/blob/60b79e2d/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java
index cc257d0..1499788 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java
@@ -17,8 +17,6 @@
  */
 package org.apache.hadoop.hbase.master;
 
-import javax.annotation.Nonnull;
-import javax.annotation.Nullable;
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
@@ -27,6 +25,8 @@ import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import com.google.common.collect.Sets;
+import edu.umd.cs.findbugs.annotations.NonNull;
+import edu.umd.cs.findbugs.annotations.Nullable;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HTableDescriptor;
@@ -167,7 +167,7 @@ public class TableStateManager {
     return rv;
   }
 
-  @Nonnull
+  @NonNull
   public TableState.State getTableState(TableName tableName) throws IOException {
     TableState currentState = readMetaState(tableName);
     if (currentState == null) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/60b79e2d/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java
index d0aeb6c..1c59a44 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java
@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.hbase.util;
 
-import javax.annotation.Nullable;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.Comparator;
@@ -30,6 +29,7 @@ import java.util.regex.Pattern;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.primitives.Ints;
+import edu.umd.cs.findbugs.annotations.Nullable;
 import org.apache.commons.lang.NotImplementedException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;

http://git-wip-us.apache.org/repos/asf/hbase/blob/60b79e2d/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
index a6dc59f..04afb01 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
@@ -47,8 +47,7 @@ import java.util.UUID;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
 
-import javax.annotation.Nullable;
-
+import edu.umd.cs.findbugs.annotations.Nullable;
 import org.apache.commons.lang.RandomStringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;

http://git-wip-us.apache.org/repos/asf/hbase/blob/60b79e2d/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
index 8e87ceb..73e5abb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
@@ -29,8 +29,7 @@ import java.util.Collection;
 import java.util.List;
 import java.util.concurrent.ExecutorService;
 
-import javax.annotation.Nullable;
-
+import edu.umd.cs.findbugs.annotations.Nullable;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -459,4 +458,4 @@ public class TestMetaWithReplicas {
     hbck = doFsck(TEST_UTIL.getConfiguration(), false);
     assertErrors(hbck, new ERROR_CODE[]{});
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/60b79e2d/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableSnapshotInputFormat.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableSnapshotInputFormat.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableSnapshotInputFormat.java
index 6285ca1..19e7a18 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableSnapshotInputFormat.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableSnapshotInputFormat.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.mapreduce;
 import com.google.common.base.Function;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Multimaps;
+import edu.umd.cs.findbugs.annotations.Nullable;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Scan;
@@ -35,7 +36,6 @@ import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.experimental.categories.Category;
 
-import javax.annotation.Nullable;
 import java.io.IOException;
 import java.util.Collection;
 import java.util.List;

http://git-wip-us.apache.org/repos/asf/hbase/blob/60b79e2d/hbase-spark/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-spark/pom.xml b/hbase-spark/pom.xml
index 89ed186..842ff21 100644
--- a/hbase-spark/pom.xml
+++ b/hbase-spark/pom.xml
@@ -62,6 +62,10 @@
             <version>${scala.version}</version>
             <scope>provided</scope>
         </dependency>
+        <!-- we exclude jsr305 below and then expressly relist it as
+             provided / optional to avoid dependency resolution possibly
+             bringing it back into runtime scope.
+        -->
         <dependency>
             <groupId>org.apache.spark</groupId>
             <artifactId>spark-core_${scala.binary.version}</artifactId>
@@ -78,9 +82,20 @@
                     <groupId>org.scala-lang</groupId>
                     <artifactId>scalap</artifactId>
                 </exclusion>
+              <exclusion>
+                <groupId>com.google.code.findbugs</groupId>
+                <artifactId>jsr305</artifactId>
+              </exclusion>
             </exclusions>
         </dependency>
         <dependency>
+          <groupId>com.google.code.findbugs</groupId>
+          <artifactId>jsr305</artifactId>
+          <version>1.3.9</version>
+          <scope>provided</scope>
+          <optional>true</optional>
+        </dependency>
+        <dependency>
             <groupId>org.apache.spark</groupId>
             <artifactId>spark-sql_${scala.binary.version}</artifactId>
             <version>${spark.version}</version>
@@ -181,6 +196,10 @@
                     <groupId>io.netty</groupId>
                     <artifactId>netty</artifactId>
                 </exclusion>
+                <exclusion>
+                  <groupId>com.google.code.findbugs</groupId>
+                  <artifactId>jsr305</artifactId>
+                </exclusion>
             </exclusions>
         </dependency>
 
@@ -215,6 +234,10 @@
                     <groupId>io.netty</groupId>
                     <artifactId>netty</artifactId>
                 </exclusion>
+                <exclusion>
+                  <groupId>com.google.code.findbugs</groupId>
+                  <artifactId>jsr305</artifactId>
+                </exclusion>
             </exclusions>
         </dependency>
 
@@ -646,6 +669,22 @@
                     </execution>
                 </executions>
             </plugin>
+      <!-- purposefully have jsr 305 exclusion only warn in this module -->
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-enforcer-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>banned-jsr305</id>
+            <goals>
+              <goal>enforce</goal>
+            </goals>
+            <configuration>
+              <fail>false</fail>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
         </plugins>
     </build>
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/60b79e2d/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index b116ae4..c4f1b05 100644
--- a/pom.xml
+++ b/pom.xml
@@ -931,36 +931,52 @@
           </dependency>
         </dependencies>
         <!-- version set by parent -->
-        <configuration>
-          <rules>
-            <!-- The earliest maven version we verify builds for via ASF Jenkins -->
-            <requireMavenVersion>
-              <version>[${maven.min.version},)</version>
-              <message>Maven is out of date.
+        <executions>
+          <execution>
+            <id>min-maven-and-java</id>
+            <goals>
+              <goal>enforce</goal>
+            </goals>
+            <configuration>
+              <rules>
+                <!-- The earliest maven version we verify builds for via ASF Jenkins -->
+                <requireMavenVersion>
+                  <version>[${maven.min.version},)</version>
+                  <message>Maven is out of date.
   HBase requires at least version ${maven.min.version} of Maven to properly build from source.
   You appear to be using an older version. You can use either "mvn -version" or
   "mvn enforcer:display-info" to verify what version is active.
   See the reference guide on building for more information: http://hbase.apache.org/book.html#build
-              </message>
-            </requireMavenVersion>
-            <!-- The earliest JVM version we verify builds for via ASF Jenkins -->
-            <requireJavaVersion>
-              <version>[${java.min.version},)</version>
-              <message>Java is out of date.
+                  </message>
+                </requireMavenVersion>
+                <!-- The earliest JVM version we verify builds for via ASF Jenkins -->
+                <requireJavaVersion>
+                  <version>[${java.min.version},)</version>
+                  <message>Java is out of date.
   HBase requirs at least version ${java.min.version} of the JDK to properly build from source.
   You appear to be using an older version. You can use either "mvn -version" or
   "mvn enforcer:display-info" to verify what version is active.
   See the reference guide on building for more information: http://hbase.apache.org/book.html#build
-              </message>
-            </requireJavaVersion>
-          </rules>
-        </configuration>
-        <executions>
+                  </message>
+                </requireJavaVersion>
+              </rules>
+            </configuration>
+          </execution>
           <execution>
-            <id>enforce</id>
+            <id>banned-jsr305</id>
             <goals>
               <goal>enforce</goal>
             </goals>
+            <configuration>
+              <rules>
+                <bannedDependencies>
+                  <excludes>
+                    <exclude>com.google.code.findbugs:jsr305</exclude>
+                  </excludes>
+                  <message>We don't allow the JSR305 jar from the Findbugs project, see HBASE-16321.</message>
+                </bannedDependencies>
+              </rules>
+            </configuration>
           </execution>
         </executions>
       </plugin>
@@ -1257,7 +1273,6 @@
     <httpcore.version>4.4.4</httpcore.version>
     <metrics-core.version>3.1.2</metrics-core.version>
     <guava.version>12.0.1</guava.version>
-    <jsr305.version>1.3.9</jsr305.version>
     <jackson.version>1.9.13</jackson.version>
     <jasper.version>5.5.23</jasper.version>
     <jaxb-api.version>2.2.2</jaxb-api.version>
@@ -1508,6 +1523,12 @@
       </dependency>
       <!-- General dependencies -->
       <dependency>
+        <groupId>com.github.stephenc.findbugs</groupId>
+        <artifactId>findbugs-annotations</artifactId>
+        <version>${findbugs-annotations}</version>
+      </dependency>
+      <!-- General dependencies -->
+      <dependency>
         <groupId>org.codehaus.jettison</groupId>
         <artifactId>jettison</artifactId>
         <version>${jettison.version}</version>
@@ -1533,11 +1554,12 @@
         <groupId>com.google.guava</groupId>
         <artifactId>guava</artifactId>
         <version>${guava.version}</version>
-      </dependency>
-      <dependency>
-         <groupId>com.google.code.findbugs</groupId>
-         <artifactId>jsr305</artifactId>
-         <version>${jsr305.version}</version>
+        <exclusions>
+          <exclusion>
+            <groupId>com.google.code.findbugs</groupId>
+            <artifactId>jsr305</artifactId>
+          </exclusion>
+        </exclusions>
       </dependency>
       <dependency>
         <groupId>commons-collections</groupId>
@@ -1853,7 +1875,6 @@
     <dependency>
       <groupId>com.github.stephenc.findbugs</groupId>
       <artifactId>findbugs-annotations</artifactId>
-      <version>${findbugs-annotations}</version>
       <scope>compile</scope>
     </dependency>
     <dependency>
@@ -2213,6 +2234,10 @@
                 <groupId>io.netty</groupId>
                 <artifactId>netty</artifactId>
               </exclusion>
+              <exclusion>
+                <groupId>com.google.code.findbugs</groupId>
+                <artifactId>jsr305</artifactId>
+              </exclusion>
             </exclusions>
           </dependency>
           <dependency>
@@ -2247,6 +2272,10 @@
                 <groupId>io.netty</groupId>
                 <artifactId>netty</artifactId>
               </exclusion>
+              <exclusion>
+                <groupId>com.google.code.findbugs</groupId>
+                <artifactId>jsr305</artifactId>
+              </exclusion>
             </exclusions>
           </dependency>
           <dependency>
@@ -2387,6 +2416,10 @@
                <groupId>io.netty</groupId>
                <artifactId>netty</artifactId>
              </exclusion>
+             <exclusion>
+               <groupId>com.google.code.findbugs</groupId>
+               <artifactId>jsr305</artifactId>
+             </exclusion>
            </exclusions>
          </dependency>
          <dependency>
@@ -2426,6 +2459,10 @@
                <groupId>io.netty</groupId>
                <artifactId>netty</artifactId>
              </exclusion>
+             <exclusion>
+               <groupId>com.google.code.findbugs</groupId>
+               <artifactId>jsr305</artifactId>
+             </exclusion>
            </exclusions>
          </dependency>
          <dependency>

http://git-wip-us.apache.org/repos/asf/hbase/blob/60b79e2d/src/main/asciidoc/_chapters/developer.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/developer.adoc b/src/main/asciidoc/_chapters/developer.adoc
index 56aa557..0a29864 100644
--- a/src/main/asciidoc/_chapters/developer.adoc
+++ b/src/main/asciidoc/_chapters/developer.adoc
@@ -1591,7 +1591,9 @@ value="HE_EQUALS_USE_HASHCODE",
 justification="I know what I'm doing")
 ----
 
-It is important to use the Apache-licensed version of the annotations.
+It is important to use the Apache-licensed version of the annotations. That generally means using
+annotations in the `edu.umd.cs.findbugs.annotations` package so that we can rely on the cleanroom
+reimplementation rather than annotations in the `javax.annotations` package.
 
 [[common.patch.feedback.javadoc.defaults]]
 ===== Javadoc - Useless Defaults


[24/50] [abbrv] hbase git commit: HBASE-16267 Revert due to failure of TestLogLevel

Posted by sy...@apache.org.
HBASE-16267 Revert due to failure of TestLogLevel


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

Branch: refs/heads/hbase-12439
Commit: 4e08a8bec9ef807134392ed05d7f0fd6eb0b1f38
Parents: 977858c
Author: tedyu <yu...@gmail.com>
Authored: Sat Aug 6 09:27:54 2016 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Sat Aug 6 09:27:54 2016 -0700

----------------------------------------------------------------------
 hbase-rest/pom.xml                              |   4 -
 .../hadoop/hbase/rest/MultiRowResource.java     |   2 +-
 .../apache/hadoop/hbase/rest/client/Client.java | 210 +++++++------------
 .../hadoop/hbase/rest/client/Response.java      |  21 +-
 .../hbase/rest/TestGetAndPutResource.java       |   4 +-
 .../hadoop/hbase/rest/TestGzipFilter.java       |  21 +-
 .../hadoop/hbase/rest/TestMultiRowResource.java |   5 +-
 .../hadoop/hbase/rest/TestScannerResource.java  |   2 +-
 .../hadoop/hbase/rest/TestSchemaResource.java   |   5 +-
 .../hbase/rest/client/TestRemoteTable.java      |  11 +-
 pom.xml                                         |  18 +-
 11 files changed, 107 insertions(+), 196 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/4e08a8be/hbase-rest/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-rest/pom.xml b/hbase-rest/pom.xml
index 4d42a7a..2e1b245 100644
--- a/hbase-rest/pom.xml
+++ b/hbase-rest/pom.xml
@@ -279,10 +279,6 @@
       <artifactId>httpclient</artifactId>
     </dependency>
     <dependency>
-      <groupId>org.apache.httpcomponents</groupId>
-      <artifactId>httpcore</artifactId>
-    </dependency>
-    <dependency>
       <groupId>commons-lang</groupId>
       <artifactId>commons-lang</artifactId>
     </dependency>

http://git-wip-us.apache.org/repos/asf/hbase/blob/4e08a8be/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/MultiRowResource.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/MultiRowResource.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/MultiRowResource.java
index b952c00..9cafe27 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/MultiRowResource.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/MultiRowResource.java
@@ -115,7 +115,7 @@ public class MultiRowResource extends ResourceBase implements Constants {
         servlet.getMetrics().incrementSucessfulGetRequests(1);
         return Response.ok(model).build();
       }
-    } catch (IOException e) {
+    } catch (Exception e) {
       servlet.getMetrics().incrementFailedGetRequests(1);
       return processException(e);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/4e08a8be/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/Client.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/Client.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/Client.java
index 204f688..e26de63 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/Client.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/Client.java
@@ -19,34 +19,30 @@
 
 package org.apache.hadoop.hbase.rest.client;
 
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
-import java.net.URI;
-import java.net.URISyntaxException;
 import java.util.Collections;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 
+import org.apache.commons.httpclient.Header;
+import org.apache.commons.httpclient.HttpClient;
+import org.apache.commons.httpclient.HttpMethod;
+import org.apache.commons.httpclient.HttpVersion;
+import org.apache.commons.httpclient.MultiThreadedHttpConnectionManager;
+import org.apache.commons.httpclient.URI;
+import org.apache.commons.httpclient.methods.ByteArrayRequestEntity;
+import org.apache.commons.httpclient.methods.DeleteMethod;
+import org.apache.commons.httpclient.methods.GetMethod;
+import org.apache.commons.httpclient.methods.HeadMethod;
+import org.apache.commons.httpclient.methods.PostMethod;
+import org.apache.commons.httpclient.methods.PutMethod;
+import org.apache.commons.httpclient.params.HttpClientParams;
+import org.apache.commons.httpclient.params.HttpConnectionManagerParams;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
-import org.apache.http.Header;
-import org.apache.http.HttpResponse;
-import org.apache.http.client.HttpClient;
-import org.apache.http.client.methods.HttpDelete;
-import org.apache.http.client.methods.HttpGet;
-import org.apache.http.client.methods.HttpHead;
-import org.apache.http.client.methods.HttpPost;
-import org.apache.http.client.methods.HttpPut;
-import org.apache.http.client.methods.HttpUriRequest;
-import org.apache.http.entity.InputStreamEntity;
-import org.apache.http.impl.client.DefaultHttpClient;
-import org.apache.http.message.BasicHeader;
-import org.apache.http.params.CoreConnectionPNames;
-import org.apache.http.util.EntityUtils;
 
 /**
  * A wrapper around HttpClient which provides some useful function and
@@ -62,8 +58,6 @@ public class Client {
   private HttpClient httpClient;
   private Cluster cluster;
   private boolean sslEnabled;
-  private HttpResponse resp;
-  private HttpGet httpGet = null;
 
   private Map<String, String> extraHeaders;
 
@@ -77,13 +71,18 @@ public class Client {
   private void initialize(Cluster cluster, boolean sslEnabled) {
     this.cluster = cluster;
     this.sslEnabled = sslEnabled;
+    MultiThreadedHttpConnectionManager manager =
+      new MultiThreadedHttpConnectionManager();
+    HttpConnectionManagerParams managerParams = manager.getParams();
+    managerParams.setConnectionTimeout(2000); // 2 s
+    managerParams.setDefaultMaxConnectionsPerHost(10);
+    managerParams.setMaxTotalConnections(100);
     extraHeaders = new ConcurrentHashMap<String, String>();
-    String clspath = System.getProperty("java.class.path");
-    LOG.debug("classpath " + clspath);
-    this.httpClient = new DefaultHttpClient();
-    this.httpClient.getParams().setIntParameter(CoreConnectionPNames.CONNECTION_TIMEOUT, 2000);
-  }
+    this.httpClient = new HttpClient(manager);
+    HttpClientParams clientParams = httpClient.getParams();
+    clientParams.setVersion(HttpVersion.HTTP_1_1);
 
+  }
   /**
    * Constructor
    * @param cluster the cluster definition
@@ -105,6 +104,9 @@ public class Client {
    * Shut down the client. Close any open persistent connections.
    */
   public void shutdown() {
+    MultiThreadedHttpConnectionManager manager =
+      (MultiThreadedHttpConnectionManager) httpClient.getHttpConnectionManager();
+    manager.shutdown();
   }
 
   /**
@@ -157,7 +159,7 @@ public class Client {
    * @return the HTTP response code
    * @throws IOException
    */
-  public HttpResponse executePathOnly(Cluster cluster, HttpUriRequest method,
+  public int executePathOnly(Cluster cluster, HttpMethod method,
       Header[] headers, String path) throws IOException {
     IOException lastException;
     if (cluster.nodes.size() < 1) {
@@ -176,29 +178,10 @@ public class Client {
         }
         sb.append(cluster.lastHost);
         sb.append(path);
-        URI uri = new URI(sb.toString());
-        if (method instanceof HttpPut) {
-          HttpPut put = new HttpPut(uri);
-          put.setEntity(((HttpPut) method).getEntity());
-          put.setHeaders(method.getAllHeaders());
-          method = put;
-        } else if (method instanceof HttpGet) {
-          method = new HttpGet(uri);
-        } else if (method instanceof HttpHead) {
-          method = new HttpHead(uri);
-        } else if (method instanceof HttpDelete) {
-          method = new HttpDelete(uri);
-        } else if (method instanceof HttpPost) {
-          HttpPost post = new HttpPost(uri);
-          post.setEntity(((HttpPost) method).getEntity());
-          post.setHeaders(method.getAllHeaders());
-          method = post;
-        }
+        URI uri = new URI(sb.toString(), true);
         return executeURI(method, headers, uri.toString());
       } catch (IOException e) {
         lastException = e;
-      } catch (URISyntaxException use) {
-        lastException = new IOException(use);
       }
     } while (++i != start && i < cluster.nodes.size());
     throw lastException;
@@ -212,27 +195,25 @@ public class Client {
    * @return the HTTP response code
    * @throws IOException
    */
-  public HttpResponse executeURI(HttpUriRequest method, Header[] headers, String uri)
+  public int executeURI(HttpMethod method, Header[] headers, String uri)
       throws IOException {
-    // method.setURI(new URI(uri, true));
+    method.setURI(new URI(uri, true));
     for (Map.Entry<String, String> e: extraHeaders.entrySet()) {
-      method.addHeader(e.getKey(), e.getValue());
+      method.addRequestHeader(e.getKey(), e.getValue());
     }
     if (headers != null) {
       for (Header header: headers) {
-        method.addHeader(header);
+        method.addRequestHeader(header);
       }
     }
     long startTime = System.currentTimeMillis();
-    if (resp != null) EntityUtils.consumeQuietly(resp.getEntity());
-    resp = httpClient.execute(method);
-
+    int code = httpClient.executeMethod(method);
     long endTime = System.currentTimeMillis();
     if (LOG.isTraceEnabled()) {
-      LOG.trace(method.getMethod() + " " + uri + " " + resp.getStatusLine().getStatusCode() + " " +
-          resp.getStatusLine().getReasonPhrase() + " in " + (endTime - startTime) + " ms");
+      LOG.trace(method.getName() + " " + uri + " " + code + " " +
+        method.getStatusText() + " in " + (endTime - startTime) + " ms");
     }
-    return resp;
+    return code;
   }
 
   /**
@@ -246,7 +227,7 @@ public class Client {
    * @return the HTTP response code
    * @throws IOException
    */
-  public HttpResponse execute(Cluster cluster, HttpUriRequest method, Header[] headers,
+  public int execute(Cluster cluster, HttpMethod method, Header[] headers,
       String path) throws IOException {
     if (path.startsWith("/")) {
       return executePathOnly(cluster, method, headers, path);
@@ -288,10 +269,11 @@ public class Client {
    */
   public Response head(Cluster cluster, String path, Header[] headers)
       throws IOException {
-    HttpHead method = new HttpHead(path);
+    HeadMethod method = new HeadMethod();
     try {
-      HttpResponse resp = execute(cluster, method, null, path);
-      return new Response(resp.getStatusLine().getStatusCode(), resp.getAllHeaders(), null);
+      int code = execute(cluster, method, null, path);
+      headers = method.getResponseHeaders();
+      return new Response(code, headers, null);
     } finally {
       method.releaseConnection();
     }
@@ -340,7 +322,7 @@ public class Client {
   public Response get(Cluster cluster, String path, String accept)
       throws IOException {
     Header[] headers = new Header[1];
-    headers[0] = new BasicHeader("Accept", accept);
+    headers[0] = new Header("Accept", accept);
     return get(cluster, path, headers);
   }
 
@@ -357,45 +339,6 @@ public class Client {
   }
 
   /**
-   * Returns the response body of the HTTPResponse, if any, as an array of bytes.
-   * If response body is not available or cannot be read, returns <tt>null</tt>
-   *
-   * Note: This will cause the entire response body to be buffered in memory. A
-   * malicious server may easily exhaust all the VM memory. It is strongly
-   * recommended, to use getResponseAsStream if the content length of the response
-   * is unknown or reasonably large.
-   *
-   * @param resp HttpResponse
-   * @return The response body, null if body is empty
-   * @throws IOException If an I/O (transport) problem occurs while obtaining the
-   * response body.
-   */
-  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value =
-      "NP_LOAD_OF_KNOWN_NULL_VALUE", justification = "null is possible return value")
-  public static byte[] getResponseBody(HttpResponse resp) throws IOException {
-    if (resp.getEntity() == null) return null;
-    try (InputStream instream = resp.getEntity().getContent()) {
-      if (instream != null) {
-        long contentLength = resp.getEntity().getContentLength();
-        if (contentLength > Integer.MAX_VALUE) {
-          //guard integer cast from overflow
-          throw new IOException("Content too large to be buffered: " + contentLength +" bytes");
-        }
-        ByteArrayOutputStream outstream = new ByteArrayOutputStream(
-            contentLength > 0 ? (int) contentLength : 4*1024);
-        byte[] buffer = new byte[4096];
-        int len;
-        while ((len = instream.read(buffer)) > 0) {
-          outstream.write(buffer, 0, len);
-        }
-        outstream.close();
-        return outstream.toByteArray();
-      }
-      return null;
-    }
-  }
-
-  /**
    * Send a GET request
    * @param c the cluster definition
    * @param path the path or URI
@@ -405,13 +348,16 @@ public class Client {
    */
   public Response get(Cluster c, String path, Header[] headers)
       throws IOException {
-    if (httpGet != null) {
-      httpGet.releaseConnection();
+    GetMethod method = new GetMethod();
+    try {
+      int code = execute(c, method, headers, path);
+      headers = method.getResponseHeaders();
+      byte[] body = method.getResponseBody();
+      InputStream in = method.getResponseBodyAsStream();
+      return new Response(code, headers, body, in);
+    } finally {
+      method.releaseConnection();
     }
-    httpGet = new HttpGet(path);
-    HttpResponse resp = execute(c, httpGet, headers, path);
-    return new Response(resp.getStatusLine().getStatusCode(), resp.getAllHeaders(),
-        resp, resp.getEntity() == null ? null : resp.getEntity().getContent());
   }
 
   /**
@@ -453,7 +399,7 @@ public class Client {
   public Response put(Cluster cluster, String path, String contentType,
       byte[] content) throws IOException {
     Header[] headers = new Header[1];
-    headers[0] = new BasicHeader("Content-Type", contentType);
+    headers[0] = new Header("Content-Type", contentType);
     return put(cluster, path, headers, content);
   }
 
@@ -471,7 +417,7 @@ public class Client {
       byte[] content, Header extraHdr) throws IOException {
     int cnt = extraHdr == null ? 1 : 2;
     Header[] headers = new Header[cnt];
-    headers[0] = new BasicHeader("Content-Type", contentType);
+    headers[0] = new Header("Content-Type", contentType);
     if (extraHdr != null) {
       headers[1] = extraHdr;
     }
@@ -504,13 +450,13 @@ public class Client {
    */
   public Response put(Cluster cluster, String path, Header[] headers,
       byte[] content) throws IOException {
-    HttpPut method = new HttpPut(path);
+    PutMethod method = new PutMethod();
     try {
-      method.setEntity(new InputStreamEntity(new ByteArrayInputStream(content), content.length));
-      HttpResponse resp = execute(cluster, method, headers, path);
-      headers = resp.getAllHeaders();
-      content = getResponseBody(resp);
-      return new Response(resp.getStatusLine().getStatusCode(), headers, content);
+      method.setRequestEntity(new ByteArrayRequestEntity(content));
+      int code = execute(cluster, method, headers, path);
+      headers = method.getResponseHeaders();
+      content = method.getResponseBody();
+      return new Response(code, headers, content);
     } finally {
       method.releaseConnection();
     }
@@ -555,7 +501,7 @@ public class Client {
   public Response post(Cluster cluster, String path, String contentType,
       byte[] content) throws IOException {
     Header[] headers = new Header[1];
-    headers[0] = new BasicHeader("Content-Type", contentType);
+    headers[0] = new Header("Content-Type", contentType);
     return post(cluster, path, headers, content);
   }
 
@@ -573,7 +519,7 @@ public class Client {
       byte[] content, Header extraHdr) throws IOException {
     int cnt = extraHdr == null ? 1 : 2;
     Header[] headers = new Header[cnt];
-    headers[0] = new BasicHeader("Content-Type", contentType);
+    headers[0] = new Header("Content-Type", contentType);
     if (extraHdr != null) {
       headers[1] = extraHdr;
     }
@@ -606,13 +552,13 @@ public class Client {
    */
   public Response post(Cluster cluster, String path, Header[] headers,
       byte[] content) throws IOException {
-    HttpPost method = new HttpPost(path);
+    PostMethod method = new PostMethod();
     try {
-      method.setEntity(new InputStreamEntity(new ByteArrayInputStream(content), content.length));
-      HttpResponse resp = execute(cluster, method, headers, path);
-      headers = resp.getAllHeaders();
-      content = getResponseBody(resp);
-      return new Response(resp.getStatusLine().getStatusCode(), headers, content);
+      method.setRequestEntity(new ByteArrayRequestEntity(content));
+      int code = execute(cluster, method, headers, path);
+      headers = method.getResponseHeaders();
+      content = method.getResponseBody();
+      return new Response(code, headers, content);
     } finally {
       method.releaseConnection();
     }
@@ -647,12 +593,12 @@ public class Client {
    * @throws IOException for error
    */
   public Response delete(Cluster cluster, String path) throws IOException {
-    HttpDelete method = new HttpDelete(path);
+    DeleteMethod method = new DeleteMethod();
     try {
-      HttpResponse resp = execute(cluster, method, null, path);
-      Header[] headers = resp.getAllHeaders();
-      byte[] content = getResponseBody(resp);
-      return new Response(resp.getStatusLine().getStatusCode(), headers, content);
+      int code = execute(cluster, method, null, path);
+      Header[] headers = method.getResponseHeaders();
+      byte[] content = method.getResponseBody();
+      return new Response(code, headers, content);
     } finally {
       method.releaseConnection();
     }
@@ -666,13 +612,13 @@ public class Client {
    * @throws IOException for error
    */
   public Response delete(Cluster cluster, String path, Header extraHdr) throws IOException {
-    HttpDelete method = new HttpDelete(path);
+    DeleteMethod method = new DeleteMethod();
     try {
       Header[] headers = { extraHdr };
-      HttpResponse resp = execute(cluster, method, headers, path);
-      headers = resp.getAllHeaders();
-      byte[] content = getResponseBody(resp);
-      return new Response(resp.getStatusLine().getStatusCode(), headers, content);
+      int code = execute(cluster, method, headers, path);
+      headers = method.getResponseHeaders();
+      byte[] content = method.getResponseBody();
+      return new Response(code, headers, content);
     } finally {
       method.releaseConnection();
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/4e08a8be/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/Response.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/Response.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/Response.java
index 27db365..871b646 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/Response.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/Response.java
@@ -19,14 +19,11 @@
 
 package org.apache.hadoop.hbase.rest.client;
 
-import java.io.IOException;
 import java.io.InputStream;
 
-import org.apache.http.Header;
-import org.apache.http.HttpResponse;
+import org.apache.commons.httpclient.Header;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
-import org.mortbay.log.Log;
 
 /**
  * The HTTP result code, response headers, and body of a HTTP response.
@@ -37,7 +34,6 @@ public class Response {
   private int code;
   private Header[] headers;
   private byte[] body;
-  private HttpResponse resp;
   private InputStream stream;
 
   /**
@@ -73,15 +69,13 @@ public class Response {
    * Constructor
    * @param code the HTTP response code
    * @param headers headers the HTTP response headers
-   * @param resp the response
+   * @param body the response body, can be null
    * @param in Inputstream if the response had one.
-   * Note: this is not thread-safe
    */
-  public Response(int code, Header[] headers, HttpResponse resp, InputStream in) {
+  public Response(int code, Header[] headers, byte[] body, InputStream in) {
     this.code = code;
     this.headers = headers;
-    this.body = null;
-    this.resp = resp;
+    this.body = body;
     this.stream = in;
   }
 
@@ -135,13 +129,6 @@ public class Response {
    * @return the HTTP response body
    */
   public byte[] getBody() {
-    if (body == null) {
-      try {
-        body = Client.getResponseBody(resp);
-      } catch (IOException ioe) {
-        Log.debug("encountered ioe when obtaining body", ioe);
-      }
-    }
     return body;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/4e08a8be/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGetAndPutResource.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGetAndPutResource.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGetAndPutResource.java
index d6eb1b3..c6fb2ff 100644
--- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGetAndPutResource.java
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGetAndPutResource.java
@@ -24,12 +24,14 @@ import java.io.ByteArrayInputStream;
 import java.io.IOException;
 import java.io.StringWriter;
 import java.net.URLEncoder;
+import java.util.Dictionary;
 import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 
 import javax.xml.bind.JAXBException;
 
-import org.apache.http.Header;
+import org.apache.commons.httpclient.Header;
 import org.apache.hadoop.hbase.CompatibilityFactory;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.rest.client.Response;

http://git-wip-us.apache.org/repos/asf/hbase/blob/4e08a8be/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGzipFilter.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGzipFilter.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGzipFilter.java
index 5097454..42d355d 100644
--- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGzipFilter.java
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGzipFilter.java
@@ -27,8 +27,7 @@ import java.io.ByteArrayOutputStream;
 import java.util.zip.GZIPInputStream;
 import java.util.zip.GZIPOutputStream;
 
-import org.apache.http.Header;
-import org.apache.http.message.BasicHeader;
+import org.apache.commons.httpclient.Header;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
@@ -96,8 +95,8 @@ public class TestGzipFilter {
     // input side filter
 
     Header[] headers = new Header[2];
-    headers[0] = new BasicHeader("Content-Type", Constants.MIMETYPE_BINARY);
-    headers[1] = new BasicHeader("Content-Encoding", "gzip");
+    headers[0] = new Header("Content-Type", Constants.MIMETYPE_BINARY);
+    headers[1] = new Header("Content-Encoding", "gzip");
     Response response = client.put(path, headers, value_1_gzip);
     assertEquals(response.getCode(), 200);
 
@@ -111,8 +110,8 @@ public class TestGzipFilter {
 
     // output side filter
 
-    headers[0] = new BasicHeader("Accept", Constants.MIMETYPE_BINARY);
-    headers[1] = new BasicHeader("Accept-Encoding", "gzip");
+    headers[0] = new Header("Accept", Constants.MIMETYPE_BINARY);
+    headers[1] = new Header("Accept-Encoding", "gzip");
     response = client.get(path, headers);
     assertEquals(response.getCode(), 200);
     ByteArrayInputStream bis = new ByteArrayInputStream(response.getBody());
@@ -129,8 +128,8 @@ public class TestGzipFilter {
   @Test
   public void testErrorNotGzipped() throws Exception {
     Header[] headers = new Header[2];
-    headers[0] = new BasicHeader("Accept", Constants.MIMETYPE_BINARY);
-    headers[1] = new BasicHeader("Accept-Encoding", "gzip");
+    headers[0] = new Header("Accept", Constants.MIMETYPE_BINARY);
+    headers[1] = new Header("Accept-Encoding", "gzip");
     Response response = client.get("/" + TABLE + "/" + ROW_1 + "/" + COLUMN_2, headers);
     assertEquals(response.getCode(), 404);
     String contentEncoding = response.getHeader("Content-Encoding");
@@ -143,9 +142,9 @@ public class TestGzipFilter {
 
   void testScannerResultCodes() throws Exception {
     Header[] headers = new Header[3];
-    headers[0] = new BasicHeader("Content-Type", Constants.MIMETYPE_XML);
-    headers[1] = new BasicHeader("Accept", Constants.MIMETYPE_JSON);
-    headers[2] = new BasicHeader("Accept-Encoding", "gzip");
+    headers[0] = new Header("Content-Type", Constants.MIMETYPE_XML);
+    headers[1] = new Header("Accept", Constants.MIMETYPE_JSON);
+    headers[2] = new Header("Accept-Encoding", "gzip");
     Response response = client.post("/" + TABLE + "/scanner", headers,
         "<Scanner/>".getBytes());
     assertEquals(response.getCode(), 201);

http://git-wip-us.apache.org/repos/asf/hbase/blob/4e08a8be/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestMultiRowResource.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestMultiRowResource.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestMultiRowResource.java
index 84d1855..958cb15 100644
--- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestMultiRowResource.java
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestMultiRowResource.java
@@ -18,8 +18,7 @@
  */
 package org.apache.hadoop.hbase.rest;
 
-import org.apache.http.Header;
-import org.apache.http.message.BasicHeader;
+import org.apache.commons.httpclient.Header;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.*;
 import org.apache.hadoop.hbase.client.Admin;
@@ -97,7 +96,7 @@ public class TestMultiRowResource {
   public static void setUpBeforeClass() throws Exception {
     conf = TEST_UTIL.getConfiguration();
     conf.setBoolean(RESTServer.REST_CSRF_ENABLED_KEY, csrfEnabled);
-    extraHdr = new BasicHeader(RESTServer.REST_CSRF_CUSTOM_HEADER_DEFAULT, "");
+    extraHdr = new Header(RESTServer.REST_CSRF_CUSTOM_HEADER_DEFAULT, "");
     TEST_UTIL.startMiniCluster();
     REST_TEST_UTIL.startServletContainer(conf);
     context = JAXBContext.newInstance(

http://git-wip-us.apache.org/repos/asf/hbase/blob/4e08a8be/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannerResource.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannerResource.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannerResource.java
index be7ee9a..5114b11 100644
--- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannerResource.java
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannerResource.java
@@ -37,7 +37,7 @@ import javax.xml.bind.JAXBException;
 import javax.xml.bind.Marshaller;
 import javax.xml.bind.Unmarshaller;
 
-import org.apache.http.Header;
+import org.apache.commons.httpclient.Header;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;

http://git-wip-us.apache.org/repos/asf/hbase/blob/4e08a8be/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestSchemaResource.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestSchemaResource.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestSchemaResource.java
index 19fdaf0..d005445 100644
--- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestSchemaResource.java
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestSchemaResource.java
@@ -28,8 +28,7 @@ import java.util.List;
 import javax.xml.bind.JAXBContext;
 import javax.xml.bind.JAXBException;
 
-import org.apache.http.Header;
-import org.apache.http.message.BasicHeader;
+import org.apache.commons.httpclient.Header;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
@@ -87,7 +86,7 @@ public class TestSchemaResource {
   public static void setUpBeforeClass() throws Exception {
     conf = TEST_UTIL.getConfiguration();
     conf.setBoolean(RESTServer.REST_CSRF_ENABLED_KEY, csrfEnabled);
-    extraHdr = new BasicHeader(RESTServer.REST_CSRF_CUSTOM_HEADER_DEFAULT, "");
+    extraHdr = new Header(RESTServer.REST_CSRF_CUSTOM_HEADER_DEFAULT, "");
     TEST_UTIL.startMiniCluster();
     REST_TEST_UTIL.startServletContainer(conf);
     client = new Client(new Cluster().add("localhost",

http://git-wip-us.apache.org/repos/asf/hbase/blob/4e08a8be/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java
index 1ac37fa..19d0587 100644
--- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java
@@ -30,8 +30,7 @@ import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
 
-import org.apache.http.Header;
-import org.apache.http.message.BasicHeader;
+import org.apache.commons.httpclient.Header;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
@@ -514,16 +513,16 @@ public class TestRemoteTable {
     Response response = new Response(200);
     assertEquals(200, response.getCode());
     Header[] headers = new Header[2];
-    headers[0] = new BasicHeader("header1", "value1");
-    headers[1] = new BasicHeader("header2", "value2");
+    headers[0] = new Header("header1", "value1");
+    headers[1] = new Header("header2", "value2");
     response = new Response(200, headers);
     assertEquals("value1", response.getHeader("header1"));
     assertFalse(response.hasBody());
     response.setCode(404);
     assertEquals(404, response.getCode());
     headers = new Header[2];
-    headers[0] = new BasicHeader("header1", "value1.1");
-    headers[1] = new BasicHeader("header2", "value2");
+    headers[0] = new Header("header1", "value1.1");
+    headers[1] = new Header("header2", "value2");
     response.setHeaders(headers);
     assertEquals("value1.1", response.getHeader("header1"));
     response.setBody(Bytes.toBytes("body"));

http://git-wip-us.apache.org/repos/asf/hbase/blob/4e08a8be/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 71b77bb..6051ba5 100644
--- a/pom.xml
+++ b/pom.xml
@@ -1275,7 +1275,7 @@
     <disruptor.version>3.3.0</disruptor.version>
     <!-- Do not use versions earlier than 3.2.2 due to a security vulnerability -->
     <collections.version>3.2.2</collections.version>
-    <httpclient.version>4.5.2</httpclient.version>
+    <httpclient.version>4.3.6</httpclient.version>
     <httpcore.version>4.4.4</httpcore.version>
     <metrics-core.version>3.1.2</metrics-core.version>
     <guava.version>12.0.1</guava.version>
@@ -2228,10 +2228,6 @@
             <artifactId>hadoop-common</artifactId>
             <version>${hadoop-two.version}</version>
             <exclusions>
-            <exclusion>
-              <groupId>commons-httpclient</groupId>
-              <artifactId>commons-httpclient</artifactId>
-            </exclusion>
               <exclusion>
                 <groupId>javax.servlet.jsp</groupId>
                 <artifactId>jsp-api</artifactId>
@@ -2266,10 +2262,6 @@
             <artifactId>hadoop-minicluster</artifactId>
             <version>${hadoop-two.version}</version>
             <exclusions>
-            <exclusion>
-              <groupId>commons-httpclient</groupId>
-              <artifactId>commons-httpclient</artifactId>
-            </exclusion>
               <exclusion>
                 <groupId>javax.servlet.jsp</groupId>
                 <artifactId>jsp-api</artifactId>
@@ -2418,10 +2410,6 @@
            <artifactId>hadoop-common</artifactId>
            <version>${hadoop-three.version}</version>
            <exclusions>
-            <exclusion>
-              <groupId>commons-httpclient</groupId>
-              <artifactId>commons-httpclient</artifactId>
-            </exclusion>
              <exclusion>
                <groupId>javax.servlet.jsp</groupId>
                <artifactId>jsp-api</artifactId>
@@ -2461,10 +2449,6 @@
            <artifactId>hadoop-minicluster</artifactId>
            <version>${hadoop-three.version}</version>
            <exclusions>
-            <exclusion>
-              <groupId>commons-httpclient</groupId>
-              <artifactId>commons-httpclient</artifactId>
-            </exclusion>
              <exclusion>
                <groupId>javax.servlet.jsp</groupId>
                <artifactId>jsp-api</artifactId>


[49/50] [abbrv] hbase git commit: Add mention to front page of hbaseconeast

Posted by sy...@apache.org.
Add mention to front page of hbaseconeast


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

Branch: refs/heads/hbase-12439
Commit: 9219d6867fac79b1702c190f035c33f5b4df8801
Parents: 907ac1f
Author: stack <st...@apache.org>
Authored: Mon Aug 15 11:41:09 2016 -0700
Committer: stack <st...@apache.org>
Committed: Mon Aug 15 11:41:09 2016 -0700

----------------------------------------------------------------------
 src/main/site/xdoc/index.xml | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/9219d686/src/main/site/xdoc/index.xml
----------------------------------------------------------------------
diff --git a/src/main/site/xdoc/index.xml b/src/main/site/xdoc/index.xml
index 50d18c3..30bbd7f 100644
--- a/src/main/site/xdoc/index.xml
+++ b/src/main/site/xdoc/index.xml
@@ -83,6 +83,7 @@ Apache HBase is an open-source, distributed, versioned, non-relational database
 </section>
 
      <section name="News">
+       <p>September 26th, 2016 <a href="http://www.meetup.com/HBase-NYC/events/233024937/">HBaseConEast2016</a> at Google in Chelsea, NYC</p>
          <p>May 24th, 2016 <a href="http://www.hbasecon.com/">HBaseCon2016</a> at The Village, 969 Market, San Francisco</p>
        <p>June 25th, 2015 <a href="http://www.zusaar.com/event/14057003">HBase Summer Meetup 2015</a> in Tokyo</p>
        <p>May 7th, 2015 <a href="http://hbasecon.com/">HBaseCon2015</a> in San Francisco</p>


[27/50] [abbrv] hbase git commit: HBASE-16367 Race between master and region server initialization may lead to premature server abort

Posted by sy...@apache.org.
HBASE-16367 Race between master and region server initialization may lead to premature server abort


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

Branch: refs/heads/hbase-12439
Commit: 50f3c9572c7c9dcf276b19c137ad600f368743ec
Parents: e5f9df1
Author: tedyu <yu...@gmail.com>
Authored: Mon Aug 8 04:07:30 2016 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Mon Aug 8 04:07:30 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/master/HMaster.java    |  3 +++
 .../apache/hadoop/hbase/regionserver/HRegionServer.java | 12 ++++++++++++
 2 files changed, 15 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/50f3c957/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index 4e6952a..5ce056d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -40,6 +40,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
+import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
@@ -451,6 +452,7 @@ public class HMaster extends HRegionServer implements MasterServices {
 
     // Some unit tests don't need a cluster, so no zookeeper at all
     if (!conf.getBoolean("hbase.testing.nocluster", false)) {
+      setInitLatch(new CountDownLatch(1));
       activeMasterManager = new ActiveMasterManager(zooKeeper, this.serverName, this);
       int infoPort = putUpJettyServer();
       startActiveMasterManager(infoPort);
@@ -693,6 +695,7 @@ public class HMaster extends HRegionServer implements MasterServices {
     // publish cluster ID
     status.setStatus("Publishing Cluster ID in ZooKeeper");
     ZKClusterId.setClusterId(this.zooKeeper, fileSystemManager.getClusterId());
+    this.initLatch.countDown();
 
     this.serverManager = createServerManager(this);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/50f3c957/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index e03993f..ba5d669 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -55,6 +55,8 @@ import java.util.TreeSet;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
@@ -233,6 +235,7 @@ public class HRegionServer extends HasThread implements
   protected MemStoreFlusher cacheFlusher;
 
   protected HeapMemoryManager hMemManager;
+  protected CountDownLatch initLatch = null;
 
   /**
    * Cluster connection to be shared by services.
@@ -655,6 +658,10 @@ public class HRegionServer extends HasThread implements
       this.fs, this.rootDir, !canUpdateTableDescriptor(), false);
   }
 
+  protected void setInitLatch(CountDownLatch latch) {
+    this.initLatch = latch;
+  }
+
   /*
    * Returns true if configured hostname should be used
    */
@@ -799,6 +806,8 @@ public class HRegionServer extends HasThread implements
    * @throws IOException
    * @throws InterruptedException
    */
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="RV_RETURN_VALUE_IGNORED_BAD_PRACTICE",
+    justification="cluster Id znode read would give us correct response")
   private void initializeZooKeeper() throws IOException, InterruptedException {
     // Create the master address tracker, register with zk, and start it.  Then
     // block until a master is available.  No point in starting up if no master
@@ -809,6 +818,9 @@ public class HRegionServer extends HasThread implements
     // when ready.
     blockAndCheckIfStopped(this.clusterStatusTracker);
 
+    if (this.initLatch != null) {
+      this.initLatch.await(50, TimeUnit.SECONDS);
+    }
     // Retrieve clusterId
     // Since cluster status is now up
     // ID should have already been set by HMaster


[13/50] [abbrv] hbase git commit: HBASE-15461 update links to blogs posted on the Cloudera website to account for site restructuring.

Posted by sy...@apache.org.
HBASE-15461 update links to blogs posted on the Cloudera website to account for site restructuring.

Signed-off-by: Dima Spivak <ds...@cloudera.com>
Signed-off-by: Matteo Bertozzi <ma...@cloudera.com>


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

Branch: refs/heads/hbase-12439
Commit: 48af3f5f7a38fe70b3518a700ccd804c2aac022d
Parents: 431c8c9
Author: Sean Busbey <bu...@cloudera.com>
Authored: Mon Mar 14 15:23:43 2016 -0500
Committer: Sean Busbey <bu...@apache.org>
Committed: Fri Aug 5 10:36:48 2016 -0500

----------------------------------------------------------------------
 src/main/asciidoc/_chapters/configuration.adoc   | 2 +-
 src/main/asciidoc/_chapters/ops_mgt.adoc         | 2 +-
 src/main/asciidoc/_chapters/other_info.adoc      | 4 ++--
 src/main/asciidoc/_chapters/security.adoc        | 4 ++--
 src/main/asciidoc/_chapters/troubleshooting.adoc | 4 ++--
 5 files changed, 8 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/48af3f5f/src/main/asciidoc/_chapters/configuration.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/configuration.adoc b/src/main/asciidoc/_chapters/configuration.adoc
index dd253d7..3738dd9 100644
--- a/src/main/asciidoc/_chapters/configuration.adoc
+++ b/src/main/asciidoc/_chapters/configuration.adoc
@@ -340,7 +340,7 @@ Do not move to Apache HBase 0.96.x if you cannot upgrade your Hadoop. See link:h
 HBase will lose data unless it is running on an HDFS that has a durable `sync` implementation.
 DO NOT use Hadoop 0.20.2, Hadoop 0.20.203.0, and Hadoop 0.20.204.0 which DO NOT have this attribute.
 Currently only Hadoop versions 0.20.205.x or any release in excess of this version -- this includes hadoop-1.0.0 -- have a working, durable sync.
-The Cloudera blog post link:http://www.cloudera.com/blog/2012/01/an-update-on-apache-hadoop-1-0/[An
+The Cloudera blog post link:https://blog.cloudera.com/blog/2012/01/an-update-on-apache-hadoop-1-0/[An
             update on Apache Hadoop 1.0] by Charles Zedlweski has a nice exposition on how all the Hadoop versions relate.
 It's worth checking out if you are having trouble making sense of the Hadoop version morass.
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/48af3f5f/src/main/asciidoc/_chapters/ops_mgt.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/ops_mgt.adoc b/src/main/asciidoc/_chapters/ops_mgt.adoc
index 6e84237..0d184a7 100644
--- a/src/main/asciidoc/_chapters/ops_mgt.adoc
+++ b/src/main/asciidoc/_chapters/ops_mgt.adoc
@@ -437,7 +437,7 @@ Caching for the input Scan is configured via `hbase.client.scanner.caching`
 By default, CopyTable utility only copies the latest version of row cells unless `--versions=n` is explicitly specified in the command.
 ====
 
-See Jonathan Hsieh's link:http://www.cloudera.com/blog/2012/06/online-hbase-backups-with-copytable-2/[Online
+See Jonathan Hsieh's link:https://blog.cloudera.com/blog/2012/06/online-hbase-backups-with-copytable-2/[Online
           HBase Backups with CopyTable] blog post for more on `CopyTable`.
 
 [[export]]

http://git-wip-us.apache.org/repos/asf/hbase/blob/48af3f5f/src/main/asciidoc/_chapters/other_info.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/other_info.adoc b/src/main/asciidoc/_chapters/other_info.adoc
index 6143876..8bcbe0f 100644
--- a/src/main/asciidoc/_chapters/other_info.adoc
+++ b/src/main/asciidoc/_chapters/other_info.adoc
@@ -59,9 +59,9 @@ link:http://ianvarley.com/UT/MR/Varley_MastersReport_Full_2009-08-07.pdf[No Rela
 [[other.info.sites]]
 === HBase Sites
 
-link:http://www.cloudera.com/blog/category/hbase/[Cloudera's HBase Blog] has a lot of links to useful HBase information.
+link:https://blog.cloudera.com/blog/category/hbase/[Cloudera's HBase Blog] has a lot of links to useful HBase information.
 
-* link:http://www.cloudera.com/blog/2010/04/cap-confusion-problems-with-partition-tolerance/[CAP Confusion] is a relevant entry for background information on distributed storage systems.
+* link:https://blog.cloudera.com/blog/2010/04/cap-confusion-problems-with-partition-tolerance/[CAP Confusion] is a relevant entry for background information on distributed storage systems.
 
 link:http://wiki.apache.org/hadoop/HBase/HBasePresentations[HBase Wiki] has a page with a number of presentations.
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/48af3f5f/src/main/asciidoc/_chapters/security.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/security.adoc b/src/main/asciidoc/_chapters/security.adoc
index 85e503c..0ed9ba2 100644
--- a/src/main/asciidoc/_chapters/security.adoc
+++ b/src/main/asciidoc/_chapters/security.adoc
@@ -126,7 +126,7 @@ A number of properties exist to configure SPNEGO authentication for the web serv
 == Secure Client Access to Apache HBase
 
 Newer releases of Apache HBase (>= 0.92) support optional SASL authentication of clients.
-See also Matteo Bertozzi's article on link:http://www.cloudera.com/blog/2012/09/understanding-user-authentication-and-authorization-in-apache-hbase/[Understanding User Authentication and Authorization in Apache HBase].
+See also Matteo Bertozzi's article on link:https://blog.cloudera.com/blog/2012/09/understanding-user-authentication-and-authorization-in-apache-hbase/[Understanding User Authentication and Authorization in Apache HBase].
 
 This describes how to set up Apache HBase and clients for connection to secure HBase resources.
 
@@ -451,7 +451,7 @@ Substitute the keytab for HTTP for _$KEYTAB_.
 == Simple User Access to Apache HBase
 
 Newer releases of Apache HBase (>= 0.92) support optional SASL authentication of clients.
-See also Matteo Bertozzi's article on link:http://www.cloudera.com/blog/2012/09/understanding-user-authentication-and-authorization-in-apache-hbase/[Understanding User Authentication and Authorization in Apache HBase].
+See also Matteo Bertozzi's article on link:https://blog.cloudera.com/blog/2012/09/understanding-user-authentication-and-authorization-in-apache-hbase/[Understanding User Authentication and Authorization in Apache HBase].
 
 This describes how to set up Apache HBase and clients for simple user access to HBase resources.
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/48af3f5f/src/main/asciidoc/_chapters/troubleshooting.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/troubleshooting.adoc b/src/main/asciidoc/_chapters/troubleshooting.adoc
index fc9aadb..ce47423 100644
--- a/src/main/asciidoc/_chapters/troubleshooting.adoc
+++ b/src/main/asciidoc/_chapters/troubleshooting.adoc
@@ -41,7 +41,7 @@ RegionServer suicides are 'normal', as this is what they do when something goes
 For example, if ulimit and max transfer threads (the two most important initial settings, see <<ulimit>> and <<dfs.datanode.max.transfer.threads>>) aren't changed, it will make it impossible at some point for DataNodes to create new threads that from the HBase point of view is seen as if HDFS was gone.
 Think about what would happen if your MySQL database was suddenly unable to access files on your local file system, well it's the same with HBase and HDFS.
 Another very common reason to see RegionServers committing seppuku is when they enter prolonged garbage collection pauses that last longer than the default ZooKeeper session timeout.
-For more information on GC pauses, see the link:http://www.cloudera.com/blog/2011/02/avoiding-full-gcs-in-hbase-with-memstore-local-allocation-buffers-part-1/[3 part blog post] by Todd Lipcon and <<gcpause>> above.
+For more information on GC pauses, see the link:https://blog.cloudera.com/blog/2011/02/avoiding-full-gcs-in-hbase-with-memstore-local-allocation-buffers-part-1/[3 part blog post] by Todd Lipcon and <<gcpause>> above.
 
 [[trouble.log]]
 == Logs
@@ -211,7 +211,7 @@ Similarly, to enable GC logging for client processes, uncomment one of the below
 # If <FILE-PATH> is not replaced, the log file(.gc) would be generated in the HBASE_LOG_DIR .
 ----
 
-For more information on GC pauses, see the link:http://www.cloudera.com/blog/2011/02/avoiding-full-gcs-in-hbase-with-memstore-local-allocation-buffers-part-1/[3 part blog post] by Todd Lipcon and <<gcpause>> above.
+For more information on GC pauses, see the link:https://blog.cloudera.com/blog/2011/02/avoiding-full-gcs-in-hbase-with-memstore-local-allocation-buffers-part-1/[3 part blog post] by Todd Lipcon and <<gcpause>> above.
 
 [[trouble.resources]]
 == Resources


[17/50] [abbrv] hbase git commit: HBASE-16308 Contain protobuf references Gather up the pb references into a few locations only rather than have pb references distributed all about the code base.

Posted by sy...@apache.org.
HBASE-16308 Contain protobuf references Gather up the pb references into a few locations only rather than have pb references distributed all about the code base.

Purge ServiceException from Callable subclasses by pushing SE handling
up into the parent Callable class (varies by context but this is basic
patten). Allows us remove a bunch of boilerplate.
Do this in the public facing classes in particular (though if
an API has SE in it -- which a few do, this patch leaves these
untouched -- for now.) Make it so HBaseAdmin and HTable have no
direct pb imports (except for endpoint processor API).

Change a few of the HBaseAdmin calls to be retrying where comments
ask that we do retry rather than one time.

Purge TimeLimitedRpcController. Lets just have one override of RpcController.

        M hbase-client/src/main/java/org/apache/hadoop/hbase/client/AbstractRegionServerCallable.java
         Cleanup. Make it clear this is an odd class for async hbase intro.

        M hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
         Refactor of RegionServerCallable allows me clean up a bunch of
         boilerplate in here and remove protobuf references.

        M hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
          Purge protobuf references everywhere except a reference to a throw of a
          ServiceException in method checkHBaseAvailable. I deprecated it in favor
          of new available method (the SE is not actually needed)

        M hbase-client/src/main/java/org/apache/hadoop/hbase/client/PayloadCarryingServerCallable.java
         Move the RetryingTimeTracker instance in here from HTable.
         Allows me to contain tracker and remove a repeated code in HTable.

        M hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionServerCallable.java
         Clean up move set up of rpc in here rather than have it repeat in HTable.
         Allows me to remove protobuf references from a bunch of places.

    M hbase-client/src/main/java/org/apache/hadoop/hbase/client/FlushRegionCallable.java
     Make use of the push of boilerplate up into RegionServerCallable

    M hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java
    M hbase-client/src/main/java/org/apache/hadoop/hbase/client/PayloadCarryingServerCallable.java
    M hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionAdminServiceCallable.java
    M hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerWithReadReplicas.java
    M hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java
    M hbase-client/src/main/java/org/apache/hadoop/hbase/client/SecureBulkLoadClient.java
    M hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
     Move boilerplate up into superclass.

    M hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetryingTimeTracker.java
     Cleanup

    M hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/PayloadCarryingRpcController.java
    M hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
    M hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEditsReplaySink.java
    M hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java
     Factor in TimeLimitedRpcController. Just have one RpcController override.

    D hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/TimeLimitedRpcController.java
     Removed. Lets have one override of pb rpccontroller only.

    M hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
     (handleRemoteException) added
     (toText) added

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/ed87a81b
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/ed87a81b
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/ed87a81b

Branch: refs/heads/hbase-12439
Commit: ed87a81b4b61c4842c12572a47c97ae23773012f
Parents: 48af3f5
Author: stack <st...@apache.org>
Authored: Wed Aug 3 10:56:38 2016 -0700
Committer: stack <st...@apache.org>
Committed: Fri Aug 5 10:13:58 2016 -0700

----------------------------------------------------------------------
 .../client/AbstractRegionServerCallable.java    |   23 +-
 .../hadoop/hbase/client/ClientScanner.java      |    2 +-
 .../hbase/client/ClientSimpleScanner.java       |    3 +-
 .../hadoop/hbase/client/ClientSmallScanner.java |   42 +-
 .../hadoop/hbase/client/ConnectionCallable.java |   56 -
 .../hbase/client/ConnectionImplementation.java  |   40 +-
 .../hbase/client/FlushRegionCallable.java       |   26 +-
 .../apache/hadoop/hbase/client/HBaseAdmin.java  | 1110 ++++++++----------
 .../org/apache/hadoop/hbase/client/HTable.java  |  455 +++----
 .../hadoop/hbase/client/MasterCallable.java     |   37 +-
 .../hbase/client/MasterKeepAliveConnection.java |    3 +-
 .../hbase/client/MultiServerCallable.java       |   35 +-
 .../client/PayloadCarryingServerCallable.java   |   44 +-
 .../client/RegionAdminServiceCallable.java      |   54 +-
 .../hbase/client/RegionServerCallable.java      |   72 +-
 .../hbase/client/RetryingTimeTracker.java       |   12 +-
 .../hbase/client/ReversedScannerCallable.java   |    4 +-
 .../hbase/client/RpcRetryingCallable.java       |   65 +
 .../hadoop/hbase/client/RpcRetryingCaller.java  |    5 +-
 .../hbase/client/RpcRetryingCallerFactory.java  |    1 +
 .../RpcRetryingCallerWithReadReplicas.java      |   26 +-
 .../hadoop/hbase/client/ScannerCallable.java    |  140 +--
 .../hbase/client/SecureBulkLoadClient.java      |   80 +-
 .../hbase/ipc/MasterCoprocessorRpcChannel.java  |    3 +-
 .../hbase/ipc/PayloadCarryingRpcController.java |  139 ++-
 .../hbase/ipc/RegionCoprocessorRpcChannel.java  |   23 +-
 .../hbase/ipc/TimeLimitedRpcController.java     |  142 ---
 .../hadoop/hbase/protobuf/ProtobufUtil.java     |   73 +-
 .../hadoop/hbase/client/TestClientScanner.java  |    1 -
 .../apache/hadoop/hbase/HBaseIOException.java   |    3 +-
 .../apache/hadoop/hbase/util/ExceptionUtil.java |    2 +-
 .../org/apache/hadoop/hbase/ipc/RpcServer.java  |    6 +-
 .../hbase/mapreduce/LoadIncrementalHFiles.java  |   36 +-
 .../master/ExpiredMobFileCleanerChore.java      |    6 -
 .../hadoop/hbase/master/ServerManager.java      |    5 +-
 .../hadoop/hbase/mob/ExpiredMobFileCleaner.java |   12 +-
 .../hadoop/hbase/mob/mapreduce/Sweeper.java     |    6 +-
 .../hbase/regionserver/RSRpcServices.java       |   15 +-
 .../regionserver/wal/WALEditsReplaySink.java    |   43 +-
 .../RegionReplicaReplicationEndpoint.java       |   54 +-
 .../org/apache/hadoop/hbase/util/Merge.java     |   13 +-
 .../org/apache/hadoop/hbase/TestNamespace.java  |    7 +-
 .../apache/hadoop/hbase/client/TestAdmin2.java  |    8 +-
 .../hadoop/hbase/client/TestClientTimeouts.java |    7 +-
 .../org/apache/hadoop/hbase/client/TestHCM.java |   37 +-
 .../hbase/client/TestReplicaWithCluster.java    |   35 +-
 .../regionserver/TestHRegionServerBulkLoad.java |   23 +-
 .../TestHRegionServerBulkLoadWithOldClient.java |   13 +-
 ...gionServerBulkLoadWithOldSecureEndpoint.java |   27 +-
 .../hbase/spark/SparkSQLPushDownFilter.java     |    4 +-
 50 files changed, 1448 insertions(+), 1630 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/ed87a81b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AbstractRegionServerCallable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AbstractRegionServerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AbstractRegionServerCallable.java
index 7279d81..5a1f5cc 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AbstractRegionServerCallable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AbstractRegionServerCallable.java
@@ -18,8 +18,7 @@
 package org.apache.hadoop.hbase.client;
 
 import java.io.IOException;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.ServerName;
@@ -29,26 +28,15 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.util.Bytes;
 
 /**
- * Implementations call a RegionServer.
- * Passed to a {@link RpcRetryingCaller} so we retry on fail.
- * TODO: this class is actually tied to one region, because most of the paths make use of
- *       the regioninfo part of location when building requests. The only reason it works for
- *       multi-region requests (e.g. batch) is that they happen to not use the region parts.
- *       This could be done cleaner (e.g. having a generic parameter and 2 derived classes,
- *       RegionCallable and actual RegionServerCallable with ServerName.
- * @param <T> the class that the ServerCallable handles
+ * Added by HBASE-15745 Refactor of RPC classes to better accept async changes.
+ * Temporary.
  */
 @InterfaceAudience.Private
 abstract class AbstractRegionServerCallable<T> implements RetryingCallable<T> {
-  // Public because used outside of this package over in ipc.
-  private static final Log LOG = LogFactory.getLog(AbstractRegionServerCallable.class);
-
   protected final Connection connection;
   protected final TableName tableName;
   protected final byte[] row;
-
   protected HRegionLocation location;
-
   protected final static int MIN_WAIT_DEAD_SERVER = 10000;
 
   /**
@@ -127,8 +115,7 @@ abstract class AbstractRegionServerCallable<T> implements RetryingCallable<T> {
   @Override
   public void prepare(final boolean reload) throws IOException {
     // check table state if this is a retry
-    if (reload &&
-        !tableName.equals(TableName.META_TABLE_NAME) &&
+    if (reload && !tableName.equals(TableName.META_TABLE_NAME) &&
         getConnection().isTableDisabled(tableName)) {
       throw new TableNotEnabledException(tableName.getNameAsString() + " is disabled.");
     }
@@ -148,4 +135,4 @@ abstract class AbstractRegionServerCallable<T> implements RetryingCallable<T> {
    * @throws IOException When client could not be created
    */
   abstract void setClientByServiceName(ServerName serviceName) throws IOException;
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/ed87a81b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java
index cb4c714..3e676c7 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java
@@ -847,4 +847,4 @@ public abstract class ClientScanner extends AbstractClientScanner {
     Cell[] list = Arrays.copyOfRange(result.rawCells(), index, result.rawCells().length);
     return Result.create(list, result.getExists(), result.isStale(), result.isPartial());
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/ed87a81b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSimpleScanner.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSimpleScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSimpleScanner.java
index f886971..ecf083b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSimpleScanner.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSimpleScanner.java
@@ -32,7 +32,6 @@ import java.util.concurrent.ExecutorService;
  */
 @InterfaceAudience.Private
 public class ClientSimpleScanner extends ClientScanner {
-
   public ClientSimpleScanner(Configuration configuration, Scan scan, TableName name,
       ClusterConnection connection, RpcRetryingCallerFactory rpcCallerFactory,
       RpcControllerFactory rpcControllerFactory, ExecutorService pool,
@@ -50,4 +49,4 @@ public class ClientSimpleScanner extends ClientScanner {
   public Result next() throws IOException {
     return nextWithSyncCache();
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/ed87a81b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallScanner.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallScanner.java
index f9bdd55..429c4cf 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallScanner.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallScanner.java
@@ -18,8 +18,10 @@
  */
 package org.apache.hadoop.hbase.client;
 
-import com.google.common.annotations.VisibleForTesting;
-import com.google.protobuf.ServiceException;
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.util.concurrent.ExecutorService;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -29,17 +31,15 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
+import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
-import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.protobuf.ResponseConverter;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse;
 import org.apache.hadoop.hbase.util.Bytes;
 
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.util.concurrent.ExecutorService;
+import com.google.common.annotations.VisibleForTesting;
 
 /**
  * Client scanner for small scan. Generally, only one RPC is called to fetch the
@@ -185,7 +185,7 @@ public class ClientSmallScanner extends ClientSimpleScanner {
     }
 
     @Override
-    public Result[] call(int timeout) throws IOException {
+    protected Result[] call(PayloadCarryingRpcController controller) throws Exception {
       if (this.closed) return null;
       if (Thread.interrupted()) {
         throw new InterruptedIOException();
@@ -193,25 +193,17 @@ public class ClientSmallScanner extends ClientSimpleScanner {
       ScanRequest request = RequestConverter.buildScanRequest(getLocation()
           .getRegionInfo().getRegionName(), getScan(), getCaching(), true);
       ScanResponse response = null;
-      controller = controllerFactory.newController();
-      try {
-        controller.setPriority(getTableName());
-        controller.setCallTimeout(timeout);
-        response = getStub().scan(controller, request);
-        Result[] results = ResponseConverter.getResults(controller.cellScanner(),
-            response);
-        if (response.hasMoreResultsInRegion()) {
-          setHasMoreResultsContext(true);
-          setServerHasMoreResults(response.getMoreResultsInRegion());
-        } else {
-          setHasMoreResultsContext(false);
-        }
-        // We need to update result metrics since we are overriding call()
-        updateResultsMetrics(results);
-        return results;
-      } catch (ServiceException se) {
-        throw ProtobufUtil.getRemoteException(se);
+      response = getStub().scan(controller, request);
+      Result[] results = ResponseConverter.getResults(controller.cellScanner(), response);
+      if (response.hasMoreResultsInRegion()) {
+        setHasMoreResultsContext(true);
+        setServerHasMoreResults(response.getMoreResultsInRegion());
+      } else {
+        setHasMoreResultsContext(false);
       }
+      // We need to update result metrics since we are overriding call()
+      updateResultsMetrics(results);
+      return results;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/ed87a81b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionCallable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionCallable.java
deleted file mode 100644
index 3f44927..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionCallable.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/**
- * 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
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.client;
-
-import java.io.Closeable;
-import java.io.IOException;
-
-/**
- * A RetryingCallable for generic connection operations.
- * @param <V> return type
- */
-abstract class ConnectionCallable<V> implements RetryingCallable<V>, Closeable {
-  protected Connection connection;
-
-  public ConnectionCallable(final Connection connection) {
-    this.connection = connection;
-  }
-
-  @Override
-  public void prepare(boolean reload) throws IOException {
-  }
-
-  @Override
-  public void close() throws IOException {
-  }
-
-  @Override
-  public void throwable(Throwable t, boolean retrying) {
-  }
-
-  @Override
-  public String getExceptionMessageAdditionalDetail() {
-    return "";
-  }
-
-  @Override
-  public long sleep(long pause, int tries) {
-    return ConnectionUtils.getPauseTime(pause, tries);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/ed87a81b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
index 8dcda13..638050f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
@@ -20,11 +20,6 @@ package org.apache.hadoop.hbase.client;
 
 import static org.apache.hadoop.hbase.client.MetricsConnection.CLIENT_SIDE_METRICS_ENABLED_KEY;
 
-import com.google.common.annotations.VisibleForTesting;
-import com.google.protobuf.BlockingRpcChannel;
-import com.google.protobuf.RpcController;
-import com.google.protobuf.ServiceException;
-
 import java.io.Closeable;
 import java.io.IOException;
 import java.io.InterruptedIOException;
@@ -68,6 +63,7 @@ import org.apache.hadoop.hbase.exceptions.RegionMovedException;
 import org.apache.hadoop.hbase.ipc.RpcClient;
 import org.apache.hadoop.hbase.ipc.RpcClientFactory;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
@@ -95,6 +91,11 @@ import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.zookeeper.KeeperException;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.BlockingRpcChannel;
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+
 /**
  * Main implementation of {@link Connection} and {@link ClusterConnection} interfaces.
  * Encapsulates connection to zookeeper and regionservers.
@@ -933,9 +934,13 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
       this.stub = null;
     }
 
-    boolean isMasterRunning() throws ServiceException {
-      MasterProtos.IsMasterRunningResponse response =
-        this.stub.isMasterRunning(null, RequestConverter.buildIsMasterRunningRequest());
+    boolean isMasterRunning() throws IOException {
+      MasterProtos.IsMasterRunningResponse response = null;
+      try {
+        response = this.stub.isMasterRunning(null, RequestConverter.buildIsMasterRunningRequest());
+      } catch (Exception e) {
+        throw ProtobufUtil.handleRemoteException(e);
+      }
       return response != null? response.getIsMasterRunning(): false;
     }
   }
@@ -1058,14 +1063,14 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
     /**
      * Once setup, check it works by doing isMasterRunning check.
      */
-    protected abstract void isMasterRunning() throws ServiceException;
+    protected abstract void isMasterRunning() throws IOException;
 
     /**
      * Create a stub. Try once only.  It is not typed because there is no common type to
      * protobuf services nor their interfaces.  Let the caller do appropriate casting.
      * @return A stub for master services.
      */
-    private Object makeStubNoRetries() throws IOException, KeeperException, ServiceException {
+    private Object makeStubNoRetries() throws IOException, KeeperException {
       ZooKeeperKeepAliveConnection zkw;
       try {
         zkw = getKeepAliveZooKeeperWatcher();
@@ -1105,7 +1110,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
     }
 
     /**
-     * Create a stub against the master.  Retry if necessary.
+     * Create a stub against the master. Retry if necessary.
      * @return A stub to do <code>intf</code> against the master
      * @throws org.apache.hadoop.hbase.MasterNotRunningException if master is not running
      */
@@ -1121,10 +1126,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
             exceptionCaught = e;
           } catch (KeeperException e) {
             exceptionCaught = e;
-          } catch (ServiceException e) {
-            exceptionCaught = e;
           }
-
           throw new MasterNotRunningException(exceptionCaught);
         } else {
           throw new DoNotRetryIOException("Connection was closed while trying to get master");
@@ -1155,8 +1157,12 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
     }
 
     @Override
-    protected void isMasterRunning() throws ServiceException {
-      this.stub.isMasterRunning(null, RequestConverter.buildIsMasterRunningRequest());
+    protected void isMasterRunning() throws IOException {
+      try {
+        this.stub.isMasterRunning(null, RequestConverter.buildIsMasterRunningRequest());
+      } catch (Exception e) {
+        throw ProtobufUtil.handleRemoteException(e);
+      }
     }
   }
 
@@ -1701,7 +1707,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
       //  java.net.ConnectException but they're not declared. So we catch it...
       LOG.info("Master connection is not running anymore", e.getUndeclaredThrowable());
       return false;
-    } catch (ServiceException se) {
+    } catch (IOException se) {
       LOG.warn("Checking master connection", se);
       return false;
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ed87a81b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FlushRegionCallable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FlushRegionCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FlushRegionCallable.java
index 73bdb74..c7bf804 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FlushRegionCallable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FlushRegionCallable.java
@@ -27,23 +27,18 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
-import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.FlushRegionRequest;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.FlushRegionResponse;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 
-import com.google.protobuf.ServiceException;
-
 /**
  * A Callable for flushRegion() RPC.
  */
 @InterfaceAudience.Private
 public class FlushRegionCallable extends RegionAdminServiceCallable<FlushRegionResponse> {
-
   private static final Log LOG = LogFactory.getLog(FlushRegionCallable.class);
-
   private final byte[] regionName;
   private final boolean writeFlushWalMarker;
   private boolean reload;
@@ -64,18 +59,14 @@ public class FlushRegionCallable extends RegionAdminServiceCallable<FlushRegionR
   }
 
   @Override
-  public FlushRegionResponse call(int callTimeout) throws Exception {
-    return flushRegion();
-  }
-
-  @Override
   public void prepare(boolean reload) throws IOException {
     super.prepare(reload);
     this.reload = reload;
   }
 
-  private FlushRegionResponse flushRegion() throws IOException {
-    // check whether we should still do the flush to this region. If the regions are changed due
+  @Override
+  protected FlushRegionResponse call(PayloadCarryingRpcController controller) throws Exception {
+    // Check whether we should still do the flush to this region. If the regions are changed due
     // to splits or merges, etc return success
     if (!Bytes.equals(location.getRegionInfo().getRegionName(), regionName)) {
       if (!reload) {
@@ -93,13 +84,6 @@ public class FlushRegionCallable extends RegionAdminServiceCallable<FlushRegionR
 
     FlushRegionRequest request =
         RequestConverter.buildFlushRegionRequest(regionName, writeFlushWalMarker);
-
-    try {
-      PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-      controller.setPriority(tableName);
-      return stub.flushRegion(controller, request);
-    } catch (ServiceException se) {
-      throw ProtobufUtil.getRemoteException(se);
-    }
+    return stub.flushRegion(controller, request);
   }
-}
+}
\ No newline at end of file


[29/50] [abbrv] hbase git commit: HBASE-9465 Push entries to peer clusters serially

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/5cadcd59/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java
new file mode 100644
index 0000000..d4af26d
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java
@@ -0,0 +1,399 @@
+/*
+ * Copyright The Apache Software Foundation
+ *
+ * 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
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.replication;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.HTestConst;
+import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.replication.ReplicationAdmin;
+import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.ReplicationTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+@Category({ ReplicationTests.class, LargeTests.class })
+public class TestSerialReplication {
+  private static final Log LOG = LogFactory.getLog(TestSerialReplication.class);
+
+  private static Configuration conf1;
+  private static Configuration conf2;
+
+  private static HBaseTestingUtility utility1;
+  private static HBaseTestingUtility utility2;
+
+  private static final byte[] famName = Bytes.toBytes("f");
+  private static final byte[] VALUE = Bytes.toBytes("v");
+  private static final byte[] ROW = Bytes.toBytes("r");
+  private static final byte[][] ROWS = HTestConst.makeNAscii(ROW, 100);
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    conf1 = HBaseConfiguration.create();
+    conf1.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/1");
+    // smaller block size and capacity to trigger more operations
+    // and test them
+    conf1.setInt("hbase.regionserver.hlog.blocksize", 1024 * 20);
+    conf1.setInt("replication.source.size.capacity", 1024);
+    conf1.setLong("replication.source.sleepforretries", 100);
+    conf1.setInt("hbase.regionserver.maxlogs", 10);
+    conf1.setLong("hbase.master.logcleaner.ttl", 10);
+    conf1.setBoolean("dfs.support.append", true);
+    conf1.setLong(HConstants.THREAD_WAKE_FREQUENCY, 100);
+    conf1.setStrings(CoprocessorHost.USER_REGION_COPROCESSOR_CONF_KEY,
+        "org.apache.hadoop.hbase.replication.TestMasterReplication$CoprocessorCounter");
+    conf1.setLong("replication.source.per.peer.node.bandwidth", 100L);// Each WAL is 120 bytes
+    conf1.setLong("replication.source.size.capacity", 1L);
+    conf1.setLong(HConstants.REPLICATION_SERIALLY_WAITING_KEY, 1000L);
+
+    utility1 = new HBaseTestingUtility(conf1);
+    utility1.startMiniZKCluster();
+    MiniZooKeeperCluster miniZK = utility1.getZkCluster();
+    new ZooKeeperWatcher(conf1, "cluster1", null, true);
+
+    conf2 = new Configuration(conf1);
+    conf2.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/2");
+
+    utility2 = new HBaseTestingUtility(conf2);
+    utility2.setZkCluster(miniZK);
+    new ZooKeeperWatcher(conf2, "cluster2", null, true);
+
+    ReplicationAdmin admin1 = new ReplicationAdmin(conf1);
+    ReplicationPeerConfig rpc = new ReplicationPeerConfig();
+    rpc.setClusterKey(utility2.getClusterKey());
+    admin1.addPeer("1", rpc, null);
+
+    utility1.startMiniCluster(1, 3);
+    utility2.startMiniCluster(1, 1);
+
+    utility1.getHBaseAdmin().setBalancerRunning(false, true);
+  }
+
+  @Test
+  public void testRegionMoveAndFailover() throws Exception {
+    TableName tableName = TableName.valueOf("testRSFailover");
+    HTableDescriptor table = new HTableDescriptor(tableName);
+    HColumnDescriptor fam = new HColumnDescriptor(famName);
+    fam.setScope(HConstants.REPLICATION_SCOPE_SERIAL);
+    table.addFamily(fam);
+    utility1.getHBaseAdmin().createTable(table);
+    utility2.getHBaseAdmin().createTable(table);
+    try(Table t1 = utility1.getConnection().getTable(tableName);
+        Table t2 = utility2.getConnection().getTable(tableName)) {
+      LOG.info("move to 1");
+      moveRegion(t1, 1);
+      LOG.info("move to 0");
+      moveRegion(t1, 0);
+      for (int i = 10; i < 20; i++) {
+        Put put = new Put(ROWS[i]);
+        put.addColumn(famName, VALUE, VALUE);
+        t1.put(put);
+      }
+      LOG.info("move to 2");
+      moveRegion(t1, 2);
+      for (int i = 20; i < 30; i++) {
+        Put put = new Put(ROWS[i]);
+        put.addColumn(famName, VALUE, VALUE);
+        t1.put(put);
+      }
+      utility1.getHBaseCluster().abortRegionServer(2);
+      for (int i = 30; i < 40; i++) {
+        Put put = new Put(ROWS[i]);
+        put.addColumn(famName, VALUE, VALUE);
+        t1.put(put);
+      }
+
+      long start = EnvironmentEdgeManager.currentTime();
+      while (EnvironmentEdgeManager.currentTime() - start < 180000) {
+        Scan scan = new Scan();
+        scan.setCaching(100);
+        List<Cell> list = new ArrayList<>();
+        try (ResultScanner results = t2.getScanner(scan)) {
+          for (Result result : results) {
+            assertEquals(1, result.rawCells().length);
+            list.add(result.rawCells()[0]);
+          }
+        }
+        List<Integer> listOfNumbers = getRowNumbers(list);
+        LOG.info(Arrays.toString(listOfNumbers.toArray()));
+        assertIntegerList(listOfNumbers, 10, 1);
+        if (listOfNumbers.size() != 30) {
+          LOG.info("Waiting all logs pushed to slave. Expected 30 , actual " + list.size());
+          Thread.sleep(200);
+          continue;
+        }
+        return;
+      }
+      throw new Exception("Not all logs have been pushed");
+    } finally {
+      utility1.getHBaseAdmin().disableTable(tableName);
+      utility2.getHBaseAdmin().disableTable(tableName);
+      utility1.getHBaseAdmin().deleteTable(tableName);
+      utility2.getHBaseAdmin().deleteTable(tableName);
+    }
+  }
+
+  @Test
+  public void testRegionSplit() throws Exception {
+    TableName tableName = TableName.valueOf("testRegionSplit");
+    HTableDescriptor table = new HTableDescriptor(tableName);
+    HColumnDescriptor fam = new HColumnDescriptor(famName);
+    fam.setScope(HConstants.REPLICATION_SCOPE_SERIAL);
+    table.addFamily(fam);
+    utility1.getHBaseAdmin().createTable(table);
+    utility2.getHBaseAdmin().createTable(table);
+    try(Table t1 = utility1.getConnection().getTable(tableName);
+        Table t2 = utility2.getConnection().getTable(tableName)) {
+
+      for (int i = 10; i < 100; i += 10) {
+        Put put = new Put(ROWS[i]);
+        put.addColumn(famName, VALUE, VALUE);
+        t1.put(put);
+      }
+      utility1.getHBaseAdmin().split(tableName, ROWS[50]);
+      Thread.sleep(5000L);
+      for (int i = 11; i < 100; i += 10) {
+        Put put = new Put(ROWS[i]);
+        put.addColumn(famName, VALUE, VALUE);
+        t1.put(put);
+      }
+      balanceTwoRegions(t1);
+      for (int i = 12; i < 100; i += 10) {
+        Put put = new Put(ROWS[i]);
+        put.addColumn(famName, VALUE, VALUE);
+        t1.put(put);
+      }
+
+      long start = EnvironmentEdgeManager.currentTime();
+      while (EnvironmentEdgeManager.currentTime() - start < 180000) {
+        Scan scan = new Scan();
+        scan.setCaching(100);
+        List<Cell> list = new ArrayList<>();
+        try (ResultScanner results = t2.getScanner(scan)) {
+          for (Result result : results) {
+            assertEquals(1, result.rawCells().length);
+            list.add(result.rawCells()[0]);
+          }
+        }
+        List<Integer> listOfNumbers = getRowNumbers(list);
+        List<Integer> list0 = new ArrayList<>();
+        List<Integer> list1 = new ArrayList<>();
+        List<Integer> list21 = new ArrayList<>();
+        List<Integer> list22 = new ArrayList<>();
+        for (int num : listOfNumbers) {
+          if (num % 10 == 0) {
+            list0.add(num);
+          } else if (num % 10 == 1) {
+            list1.add(num);
+          } else if (num < 50) { //num%10==2
+            list21.add(num);
+          } else { // num%10==1&&num>50
+            list22.add(num);
+          }
+        }
+
+        LOG.info(Arrays.toString(list0.toArray()));
+        LOG.info(Arrays.toString(list1.toArray()));
+        LOG.info(Arrays.toString(list21.toArray()));
+        LOG.info(Arrays.toString(list22.toArray()));
+        assertIntegerList(list0, 10, 10);
+        assertIntegerList(list1, 11, 10);
+        assertIntegerList(list21, 12, 10);
+        assertIntegerList(list22, 52, 10);
+        if (!list1.isEmpty()) {
+          assertEquals(9, list0.size());
+        }
+        if (!list21.isEmpty() || !list22.isEmpty()) {
+          assertEquals(9, list1.size());
+        }
+
+        if (list.size() == 27) {
+          return;
+        }
+        LOG.info("Waiting all logs pushed to slave. Expected 27 , actual " + list.size());
+        Thread.sleep(200);
+      }
+      throw new Exception("Not all logs have been pushed");
+    } finally {
+      utility1.getHBaseAdmin().disableTable(tableName);
+      utility2.getHBaseAdmin().disableTable(tableName);
+      utility1.getHBaseAdmin().deleteTable(tableName);
+      utility2.getHBaseAdmin().deleteTable(tableName);
+    }
+  }
+
+  @Test
+  public void testRegionMerge() throws Exception {
+    TableName tableName = TableName.valueOf("testRegionMerge");
+    HTableDescriptor table = new HTableDescriptor(tableName);
+    HColumnDescriptor fam = new HColumnDescriptor(famName);
+    fam.setScope(HConstants.REPLICATION_SCOPE_SERIAL);
+    table.addFamily(fam);
+    utility1.getHBaseAdmin().createTable(table);
+    utility2.getHBaseAdmin().createTable(table);
+    utility1.getHBaseAdmin().split(tableName, ROWS[50]);
+    Thread.sleep(5000L);
+
+    try(Table t1 = utility1.getConnection().getTable(tableName);
+        Table t2 = utility2.getConnection().getTable(tableName)) {
+      for (int i = 10; i < 100; i += 10) {
+        Put put = new Put(ROWS[i]);
+        put.addColumn(famName, VALUE, VALUE);
+        t1.put(put);
+      }
+      List<Pair<HRegionInfo, ServerName>> regions =
+          MetaTableAccessor.getTableRegionsAndLocations(utility1.getConnection(), tableName);
+      assertEquals(2, regions.size());
+      utility1.getHBaseAdmin().mergeRegions(regions.get(0).getFirst().getRegionName(),
+          regions.get(1).getFirst().getRegionName(), true);
+      for (int i = 11; i < 100; i += 10) {
+        Put put = new Put(ROWS[i]);
+        put.addColumn(famName, VALUE, VALUE);
+        t1.put(put);
+      }
+
+      long start = EnvironmentEdgeManager.currentTime();
+      while (EnvironmentEdgeManager.currentTime() - start < 180000) {
+        Scan scan = new Scan();
+        scan.setCaching(100);
+        List<Cell> list = new ArrayList<>();
+        try (ResultScanner results = t2.getScanner(scan)) {
+          for (Result result : results) {
+            assertEquals(1, result.rawCells().length);
+            list.add(result.rawCells()[0]);
+          }
+        }
+        List<Integer> listOfNumbers = getRowNumbers(list);
+        List<Integer> list0 = new ArrayList<>();
+        List<Integer> list1 = new ArrayList<>();
+        for (int num : listOfNumbers) {
+          if (num % 10 == 0) {
+            list0.add(num);
+          } else {
+            list1.add(num);
+          }
+        }
+        LOG.info(Arrays.toString(list0.toArray()));
+        LOG.info(Arrays.toString(list1.toArray()));
+        assertIntegerList(list0, 10, 10);
+        assertIntegerList(list1, 11, 10);
+        if (!list1.isEmpty()) {
+          assertEquals(9, list0.size());
+        }
+        if (list.size() == 18) {
+          return;
+        }
+        LOG.info("Waiting all logs pushed to slave. Expected 18 , actual " + list.size());
+        Thread.sleep(200);
+      }
+
+    } finally {
+      utility1.getHBaseAdmin().disableTable(tableName);
+      utility2.getHBaseAdmin().disableTable(tableName);
+      utility1.getHBaseAdmin().deleteTable(tableName);
+      utility2.getHBaseAdmin().deleteTable(tableName);
+    }
+  }
+
+  private List<Integer> getRowNumbers(List<Cell> cells) {
+    List<Integer> listOfRowNumbers = new ArrayList<>();
+    for (Cell c : cells) {
+      listOfRowNumbers.add(Integer.parseInt(Bytes
+          .toString(c.getRowArray(), c.getRowOffset() + ROW.length,
+              c.getRowLength() - ROW.length)));
+    }
+    return listOfRowNumbers;
+  }
+
+  @AfterClass
+  public static void setUpAfterClass() throws Exception {
+    utility2.shutdownMiniCluster();
+    utility1.shutdownMiniCluster();
+  }
+
+  private void moveRegion(Table table, int index) throws IOException {
+    List<Pair<HRegionInfo, ServerName>> regions =
+        MetaTableAccessor.getTableRegionsAndLocations(utility1.getConnection(), table.getName());
+    assertEquals(1, regions.size());
+    HRegionInfo regionInfo = regions.get(0).getFirst();
+    ServerName name = utility1.getHBaseCluster().getRegionServer(index).getServerName();
+    utility1.getAdmin()
+        .move(regionInfo.getEncodedNameAsBytes(), Bytes.toBytes(name.getServerName()));
+    try {
+      Thread.sleep(5000L); // wait to complete
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+    }
+  }
+
+  private void balanceTwoRegions(Table table) throws Exception {
+    List<Pair<HRegionInfo, ServerName>> regions =
+        MetaTableAccessor.getTableRegionsAndLocations(utility1.getConnection(), table.getName());
+    assertEquals(2, regions.size());
+    HRegionInfo regionInfo1 = regions.get(0).getFirst();
+    ServerName name1 = utility1.getHBaseCluster().getRegionServer(0).getServerName();
+    HRegionInfo regionInfo2 = regions.get(1).getFirst();
+    ServerName name2 = utility1.getHBaseCluster().getRegionServer(1).getServerName();
+    utility1.getAdmin()
+        .move(regionInfo1.getEncodedNameAsBytes(), Bytes.toBytes(name1.getServerName()));
+    Thread.sleep(5000L);
+    utility1.getAdmin()
+        .move(regionInfo2.getEncodedNameAsBytes(), Bytes.toBytes(name2.getServerName()));
+    Thread.sleep(5000L);
+  }
+
+  private void assertIntegerList(List<Integer> list, int start, int step) {
+    int size = list.size();
+    for (int i = 0; i < size; i++) {
+      assertTrue(list.get(i) == start + step * i);
+    }
+  }
+}


[09/50] [abbrv] hbase git commit: HBASE-16359 Addendum moves null checking as suggested by Heng

Posted by sy...@apache.org.
HBASE-16359 Addendum moves null checking as suggested by Heng


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

Branch: refs/heads/hbase-12439
Commit: 4b3e45fceb0d65093f6ca5d821eb1f2cb26398df
Parents: e8f08ba
Author: tedyu <yu...@gmail.com>
Authored: Fri Aug 5 03:18:28 2016 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Fri Aug 5 03:18:28 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/regionserver/RSRpcServices.java    | 16 ++++++++--------
 1 file changed, 8 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/4b3e45fc/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index 0de4a28..d87ada4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -1684,11 +1684,6 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
           throw new IOException(error);
         }
         LOG.info("Open " + region.getRegionNameAsString());
-        htd = htds.get(region.getTable());
-        if (htd == null) {
-          htd = regionServer.tableDescriptors.get(region.getTable());
-          htds.put(region.getTable(), htd);
-        }
 
         final Boolean previous = regionServer.regionsInTransitionInRS.putIfAbsent(
           encodedNameBytes, Boolean.TRUE);
@@ -1733,6 +1728,14 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
                 tmpRegions);
             }
           }
+          htd = htds.get(region.getTable());
+          if (htd == null) {
+            htd = regionServer.tableDescriptors.get(region.getTable());
+            htds.put(region.getTable(), htd);
+          }
+          if (htd == null) {
+            throw new IOException("Missing table descriptor for " + region.getEncodedName());
+          }
           // If there is no action in progress, we can submit a specific handler.
           // Need to pass the expected version in the constructor.
           if (region.isMetaRegion()) {
@@ -1741,9 +1744,6 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
           } else {
             regionServer.updateRegionFavoredNodesMapping(region.getEncodedName(),
               regionOpenInfo.getFavoredNodesList());
-            if (htd == null) {
-              throw new IOException("Missing table descriptor for " + region.getEncodedName());
-            }
             if (htd.getPriority() >= HConstants.ADMIN_QOS || region.getTable().isSystemTable()) {
               regionServer.service.submit(new OpenPriorityRegionHandler(
                 regionServer, regionServer, region, htd, masterSystemTime));


[37/50] [abbrv] hbase git commit: REVERT of revert of "HBASE-16308 Contain protobuf references Gather up the pb references into a few locations only rather than have pb references distributed all about the code base." This is a revert of a revert; i.e. w

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SecureBulkLoadClient.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SecureBulkLoadClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SecureBulkLoadClient.java
index 7b1547d..f460bdb 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SecureBulkLoadClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SecureBulkLoadClient.java
@@ -21,9 +21,11 @@ package org.apache.hadoop.hbase.client;
 import java.io.IOException;
 import java.util.List;
 
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequest;
@@ -38,41 +40,35 @@ import org.apache.hadoop.hbase.security.SecureBulkLoadUtil;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.security.token.Token;
 
-import com.google.protobuf.ServiceException;
-
 /**
  * Client proxy for SecureBulkLoadProtocol
  */
 @InterfaceAudience.Private
 public class SecureBulkLoadClient {
   private Table table;
+  private final RpcControllerFactory rpcControllerFactory;
 
-  public SecureBulkLoadClient(Table table) {
+  public SecureBulkLoadClient(final Configuration conf, Table table) {
     this.table = table;
+    this.rpcControllerFactory = new RpcControllerFactory(conf);
   }
 
   public String prepareBulkLoad(final Connection conn) throws IOException {
     try {
-      RegionServerCallable<String> callable =
-          new RegionServerCallable<String>(conn, table.getName(), HConstants.EMPTY_START_ROW) {
-            @Override
-            public String call(int callTimeout) throws IOException {
-              byte[] regionName = getLocation().getRegionInfo().getRegionName();
-              RegionSpecifier region =
-                  RequestConverter
-                      .buildRegionSpecifier(RegionSpecifierType.REGION_NAME, regionName);
-              try {
-                PrepareBulkLoadRequest request =
-                    PrepareBulkLoadRequest.newBuilder()
-                        .setTableName(ProtobufUtil.toProtoTableName(table.getName()))
-                        .setRegion(region).build();
-                PrepareBulkLoadResponse response = getStub().prepareBulkLoad(null, request);
-                return response.getBulkToken();
-              } catch (ServiceException se) {
-                throw ProtobufUtil.getRemoteException(se);
-              }
-            }
-          };
+      RegionServerCallable<String> callable = new RegionServerCallable<String>(conn,
+          this.rpcControllerFactory, table.getName(), HConstants.EMPTY_START_ROW) {
+        @Override
+        protected String rpcCall() throws Exception {
+          byte[] regionName = getLocation().getRegionInfo().getRegionName();
+          RegionSpecifier region =
+              RequestConverter.buildRegionSpecifier(RegionSpecifierType.REGION_NAME, regionName);
+          PrepareBulkLoadRequest request = PrepareBulkLoadRequest.newBuilder()
+              .setTableName(ProtobufUtil.toProtoTableName(table.getName()))
+              .setRegion(region).build();
+          PrepareBulkLoadResponse response = getStub().prepareBulkLoad(null, request);
+          return response.getBulkToken();
+        }
+      };
       return RpcRetryingCallerFactory.instantiate(conn.getConfiguration(), null)
           .<String> newCaller().callWithRetries(callable, Integer.MAX_VALUE);
     } catch (Throwable throwable) {
@@ -82,24 +78,19 @@ public class SecureBulkLoadClient {
 
   public void cleanupBulkLoad(final Connection conn, final String bulkToken) throws IOException {
     try {
-      RegionServerCallable<Void> callable =
-          new RegionServerCallable<Void>(conn, table.getName(), HConstants.EMPTY_START_ROW) {
-            @Override
-            public Void call(int callTimeout) throws IOException {
-              byte[] regionName = getLocation().getRegionInfo().getRegionName();
-              RegionSpecifier region = RequestConverter.buildRegionSpecifier(
-                RegionSpecifierType.REGION_NAME, regionName);
-              try {
-                CleanupBulkLoadRequest request =
-                    CleanupBulkLoadRequest.newBuilder().setRegion(region)
-                        .setBulkToken(bulkToken).build();
-                getStub().cleanupBulkLoad(null, request);
-              } catch (ServiceException se) {
-                throw ProtobufUtil.getRemoteException(se);
-              }
-              return null;
-            }
-          };
+      RegionServerCallable<Void> callable = new RegionServerCallable<Void>(conn,
+          this.rpcControllerFactory, table.getName(), HConstants.EMPTY_START_ROW) {
+        @Override
+        protected Void rpcCall() throws Exception {
+          byte[] regionName = getLocation().getRegionInfo().getRegionName();
+          RegionSpecifier region = RequestConverter.buildRegionSpecifier(
+              RegionSpecifierType.REGION_NAME, regionName);
+          CleanupBulkLoadRequest request =
+              CleanupBulkLoadRequest.newBuilder().setRegion(region).setBulkToken(bulkToken).build();
+          getStub().cleanupBulkLoad(null, request);
+          return null;
+        }
+      };
       RpcRetryingCallerFactory.instantiate(conn.getConfiguration(), null)
           .<Void> newCaller().callWithRetries(callable, Integer.MAX_VALUE);
     } catch (Throwable throwable) {
@@ -130,12 +121,12 @@ public class SecureBulkLoadClient {
     try {
       BulkLoadHFileResponse response = client.bulkLoadHFile(null, request);
       return response.getLoaded();
-    } catch (ServiceException se) {
-      throw ProtobufUtil.getRemoteException(se);
+    } catch (Exception se) {
+      throw ProtobufUtil.handleRemoteException(se);
     }
   }
 
   public Path getStagingPath(String bulkToken, byte[] family) throws IOException {
     return SecureBulkLoadUtil.getStagingPath(table.getConfiguration(), bulkToken, family);
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/MasterCoprocessorRpcChannel.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/MasterCoprocessorRpcChannel.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/MasterCoprocessorRpcChannel.java
index 6fae5cb..a6384e3 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/MasterCoprocessorRpcChannel.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/MasterCoprocessorRpcChannel.java
@@ -77,5 +77,4 @@ public class MasterCoprocessorRpcChannel extends SyncCoprocessorRpcChannel {
     }
     return response;
   }
-
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/PayloadCarryingRpcController.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/PayloadCarryingRpcController.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/PayloadCarryingRpcController.java
index f4f18b3..d9877dc 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/PayloadCarryingRpcController.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/PayloadCarryingRpcController.java
@@ -17,7 +17,9 @@
  */
 package org.apache.hadoop.hbase.ipc;
 
+import java.io.IOException;
 import java.util.List;
+import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.hadoop.hbase.CellScannable;
 import org.apache.hadoop.hbase.CellScanner;
@@ -26,15 +28,26 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 
+import com.google.protobuf.RpcCallback;
+import com.google.protobuf.RpcController;
+
 /**
  * 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.  This class is used ferrying data across the proxy/protobuf
- * service chasm.  Used by client and server ipc'ing.
+ * 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.
  */
 @InterfaceAudience.Private
-public class PayloadCarryingRpcController
-    extends TimeLimitedRpcController implements CellScannable {
+public class PayloadCarryingRpcController implements RpcController, CellScannable {
+  /**
+   * The time, in ms before the call should expire.
+   */
+  protected volatile Integer callTimeout;
+  protected volatile boolean cancelled = false;
+  protected final AtomicReference<RpcCallback<Object>> cancellationCb = new AtomicReference<>(null);
+  protected final AtomicReference<RpcCallback<IOException>> failureCb = new AtomicReference<>(null);
+  private IOException exception;
 
   public static final int PRIORITY_UNSET = -1;
   /**
@@ -88,8 +101,8 @@ public class PayloadCarryingRpcController
    * @param tn Set priority based off the table we are going against.
    */
   public void setPriority(final TableName tn) {
-    this.priority =
-        (tn != null && tn.isSystemTable())? HConstants.SYSTEMTABLE_QOS: HConstants.NORMAL_QOS;
+    setPriority(tn != null && tn.isSystemTable()? HConstants.SYSTEMTABLE_QOS:
+      HConstants.NORMAL_QOS);
   }
 
   /**
@@ -99,9 +112,103 @@ public class PayloadCarryingRpcController
     return priority;
   }
 
-  @Override public void reset() {
-    super.reset();
+  @Override
+  public void reset() {
     priority = 0;
     cellScanner = null;
+    exception = null;
+    cancelled = false;
+    failureCb.set(null);
+    cancellationCb.set(null);
+    callTimeout = null;
+  }
+
+  public int getCallTimeout() {
+    if (callTimeout != null) {
+      return callTimeout;
+    } else {
+      return 0;
+    }
+  }
+
+  public void setCallTimeout(int callTimeout) {
+    this.callTimeout = callTimeout;
+  }
+
+  public boolean hasCallTimeout(){
+    return callTimeout != null;
+  }
+
+  @Override
+  public String errorText() {
+    if (exception != null) {
+      return exception.getMessage();
+    } else {
+      return null;
+    }
+  }
+
+  /**
+   * For use in async rpc clients
+   * @return true if failed
+   */
+  @Override
+  public boolean failed() {
+    return this.exception != null;
+  }
+
+  @Override
+  public boolean isCanceled() {
+    return cancelled;
+  }
+
+  @Override
+  public void notifyOnCancel(RpcCallback<Object> cancellationCb) {
+    this.cancellationCb.set(cancellationCb);
+    if (this.cancelled) {
+      cancellationCb.run(null);
+    }
+  }
+
+  /**
+   * Notify a callback on error.
+   * For use in async rpc clients
+   *
+   * @param failureCb the callback to call on error
+   */
+  public void notifyOnFail(RpcCallback<IOException> failureCb) {
+    this.failureCb.set(failureCb);
+    if (this.exception != null) {
+      failureCb.run(this.exception);
+    }
+  }
+
+  @Override
+  public void setFailed(String reason) {
+    this.exception = new IOException(reason);
+    if (this.failureCb.get() != null) {
+      this.failureCb.get().run(this.exception);
+    }
+  }
+
+  /**
+   * Set failed with an exception to pass on.
+   * For use in async rpc clients
+   *
+   * @param e exception to set with
+   */
+  public void setFailed(IOException e) {
+    this.exception = e;
+    if (this.failureCb.get() != null) {
+      this.failureCb.get().run(this.exception);
+    }
+  }
+
+  @Override
+  public void startCancel() {
+    cancelled = true;
+    if (cancellationCb.get() != null) {
+      cancellationCb.get().run(null);
+    }
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionCoprocessorRpcChannel.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionCoprocessorRpcChannel.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionCoprocessorRpcChannel.java
index 55d6375..209deed 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionCoprocessorRpcChannel.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionCoprocessorRpcChannel.java
@@ -36,11 +36,12 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServic
 import org.apache.hadoop.hbase.util.Bytes;
 
 /**
- * Provides clients with an RPC connection to call coprocessor endpoint {@link com.google.protobuf.Service}s
+ * Provides clients with an RPC connection to call Coprocessor Endpoint
+ * {@link com.google.protobuf.Service}s
  * against a given table region.  An instance of this class may be obtained
  * by calling {@link org.apache.hadoop.hbase.client.Table#coprocessorService(byte[])},
- * but should normally only be used in creating a new {@link com.google.protobuf.Service} stub to call the endpoint
- * methods.
+ * but should normally only be used in creating a new {@link com.google.protobuf.Service} stub to
+ * call the endpoint methods.
  * @see org.apache.hadoop.hbase.client.Table#coprocessorService(byte[])
  */
 @InterfaceAudience.Private
@@ -76,30 +77,21 @@ public class RegionCoprocessorRpcChannel extends SyncCoprocessorRpcChannel {
       Descriptors.MethodDescriptor method, Message request, Message responsePrototype)
           throws IOException {
     if (LOG.isTraceEnabled()) {
-      LOG.trace("Call: "+method.getName()+", "+request.toString());
+      LOG.trace("Call: " + method.getName() + ", " + request.toString());
     }
-
     if (row == null) {
       throw new IllegalArgumentException("Missing row property for remote region location");
     }
-
-    final RpcController rpcController = controller == null
-        ? rpcControllerFactory.newController() : controller;
-
     final ClientProtos.CoprocessorServiceCall call =
         CoprocessorRpcUtils.buildServiceCall(row, method, request);
     RegionServerCallable<CoprocessorServiceResponse> callable =
-        new RegionServerCallable<CoprocessorServiceResponse>(connection, table, row) {
+        new RegionServerCallable<CoprocessorServiceResponse>(connection,
+          controller == null? this.rpcControllerFactory.newController(): controller,
+          table, row) {
       @Override
-      public CoprocessorServiceResponse call(int callTimeout) throws Exception {
-        if (rpcController instanceof PayloadCarryingRpcController) {
-          ((PayloadCarryingRpcController) rpcController).setPriority(tableName);
-        }
-        if (rpcController instanceof TimeLimitedRpcController) {
-          ((TimeLimitedRpcController) rpcController).setCallTimeout(callTimeout);
-        }
+      protected CoprocessorServiceResponse rpcCall() throws Exception {
         byte[] regionName = getLocation().getRegionInfo().getRegionName();
-        return ProtobufUtil.execService(rpcController, getStub(), call, regionName);
+        return ProtobufUtil.execService(getRpcController(), getStub(), call, regionName);
       }
     };
     CoprocessorServiceResponse result = rpcCallerFactory.<CoprocessorServiceResponse> newCaller()
@@ -126,4 +118,4 @@ public class RegionCoprocessorRpcChannel extends SyncCoprocessorRpcChannel {
   public byte[] getLastRegion() {
     return lastRegion;
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcControllerFactory.java
----------------------------------------------------------------------
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 faeca8d..4b84df1 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
@@ -47,6 +47,7 @@ public class RpcControllerFactory {
   }
 
   public PayloadCarryingRpcController newController() {
+    // TODO: Set HConstants default rpc timeout here rather than nothing?
     return new PayloadCarryingRpcController();
   }
 
@@ -80,4 +81,4 @@ public class RpcControllerFactory {
       return new RpcControllerFactory(configuration);
     }
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/TimeLimitedRpcController.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/TimeLimitedRpcController.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/TimeLimitedRpcController.java
deleted file mode 100644
index cf08ea9..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/TimeLimitedRpcController.java
+++ /dev/null
@@ -1,142 +0,0 @@
-/**
- * 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
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.ipc;
-
-import com.google.protobuf.RpcCallback;
-import com.google.protobuf.RpcController;
-
-import java.io.IOException;
-import java.util.concurrent.atomic.AtomicReference;
-
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-
-@InterfaceAudience.Private
-public class TimeLimitedRpcController implements RpcController {
-
-  /**
-   * The time, in ms before the call should expire.
-   */
-  protected volatile Integer callTimeout;
-  protected volatile boolean cancelled = false;
-  protected final AtomicReference<RpcCallback<Object>> cancellationCb =
-      new AtomicReference<>(null);
-
-  protected final AtomicReference<RpcCallback<IOException>> failureCb =
-      new AtomicReference<>(null);
-
-  private IOException exception;
-
-  public int getCallTimeout() {
-    if (callTimeout != null) {
-      return callTimeout;
-    } else {
-      return 0;
-    }
-  }
-
-  public void setCallTimeout(int callTimeout) {
-    this.callTimeout = callTimeout;
-  }
-
-  public boolean hasCallTimeout(){
-    return callTimeout != null;
-  }
-
-  @Override
-  public String errorText() {
-    if (exception != null) {
-      return exception.getMessage();
-    } else {
-      return null;
-    }
-  }
-
-  /**
-   * For use in async rpc clients
-   * @return true if failed
-   */
-  @Override
-  public boolean failed() {
-    return this.exception != null;
-  }
-
-  @Override
-  public boolean isCanceled() {
-    return cancelled;
-  }
-
-  @Override
-  public void notifyOnCancel(RpcCallback<Object> cancellationCb) {
-    this.cancellationCb.set(cancellationCb);
-    if (this.cancelled) {
-      cancellationCb.run(null);
-    }
-  }
-
-  /**
-   * Notify a callback on error.
-   * For use in async rpc clients
-   *
-   * @param failureCb the callback to call on error
-   */
-  public void notifyOnFail(RpcCallback<IOException> failureCb) {
-    this.failureCb.set(failureCb);
-    if (this.exception != null) {
-      failureCb.run(this.exception);
-    }
-  }
-
-  @Override
-  public void reset() {
-    exception = null;
-    cancelled = false;
-    failureCb.set(null);
-    cancellationCb.set(null);
-    callTimeout = null;
-  }
-
-  @Override
-  public void setFailed(String reason) {
-    this.exception = new IOException(reason);
-    if (this.failureCb.get() != null) {
-      this.failureCb.get().run(this.exception);
-    }
-  }
-
-  /**
-   * Set failed with an exception to pass on.
-   * For use in async rpc clients
-   *
-   * @param e exception to set with
-   */
-  public void setFailed(IOException e) {
-    this.exception = e;
-    if (this.failureCb.get() != null) {
-      this.failureCb.get().run(this.exception);
-    }
-  }
-
-  @Override
-  public void startCancel() {
-    cancelled = true;
-    if (cancellationCb.get() != null) {
-      cancellationCb.get().run(null);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
index 5ba0572..623acd5 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
@@ -17,6 +17,9 @@
  */
 package org.apache.hadoop.hbase.protobuf;
 
+import static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier
+.RegionSpecifierType.REGION_NAME;
+
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
@@ -38,14 +41,11 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.NavigableSet;
 import java.util.Set;
+import java.util.concurrent.Callable;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
-
-import static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier
-.RegionSpecifierType.REGION_NAME;
-
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.CellUtil;
@@ -53,6 +53,7 @@ import org.apache.hadoop.hbase.ClusterId;
 import org.apache.hadoop.hbase.ClusterStatus;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HBaseIOException;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
@@ -124,8 +125,8 @@ import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos;
 import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.LiveServerInfo;
 import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionInTransition;
 import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionLoad;
-import org.apache.hadoop.hbase.protobuf.generated.FSProtos.HBaseVersionFileContent;
 import org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos;
+import org.apache.hadoop.hbase.protobuf.generated.FSProtos.HBaseVersionFileContent;
 import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair;
@@ -171,11 +172,9 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.DynamicClassLoader;
 import org.apache.hadoop.hbase.util.ExceptionUtil;
 import org.apache.hadoop.hbase.util.Methods;
-import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.VersionInfo;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 
 import com.google.common.collect.ArrayListMultimap;
@@ -334,17 +333,32 @@ public final class ProtobufUtil {
    *   a new IOException that wraps the unexpected ServiceException.
    */
   public static IOException getRemoteException(ServiceException se) {
-    Throwable e = se.getCause();
-    if (e == null) {
-      return new IOException(se);
+    return makeIOExceptionOfException(se);
+  }
+
+  /**
+   * Like {@link #getRemoteException(ServiceException)} but more generic, able to handle more than
+   * just {@link ServiceException}. Prefer this method to
+   * {@link #getRemoteException(ServiceException)} because trying to
+   * contain direct protobuf references.
+   * @param e
+   */
+  public static IOException handleRemoteException(Exception e) {
+    return makeIOExceptionOfException(e);
+  }
+
+  private static IOException makeIOExceptionOfException(Exception e) {
+    Throwable t = e;
+    if (e instanceof ServiceException) {
+      t = e.getCause();
     }
-    if (ExceptionUtil.isInterrupt(e)) {
-      return ExceptionUtil.asInterrupt(e);
+    if (ExceptionUtil.isInterrupt(t)) {
+      return ExceptionUtil.asInterrupt(t);
     }
-    if (e instanceof RemoteException) {
-      e = ((RemoteException) e).unwrapRemoteException();
+    if (t instanceof RemoteException) {
+      t = ((RemoteException)t).unwrapRemoteException();
     }
-    return e instanceof IOException ? (IOException) e : new IOException(se);
+    return t instanceof IOException? (IOException)t: new HBaseIOException(t);
   }
 
   /**
@@ -1252,7 +1266,6 @@ public final class ProtobufUtil {
     return toMutation(type, mutation, builder, HConstants.NO_NONCE);
   }
 
-  @SuppressWarnings("deprecation")
   public static MutationProto toMutation(final MutationType type, final Mutation mutation,
       MutationProto.Builder builder, long nonce)
   throws IOException {
@@ -2658,13 +2671,11 @@ public final class ProtobufUtil {
     }
   }
 
-  @SuppressWarnings("deprecation")
   public static CompactionDescriptor toCompactionDescriptor(HRegionInfo info, byte[] family,
       List<Path> inputPaths, List<Path> outputPaths, Path storeDir) {
     return toCompactionDescriptor(info, null, family, inputPaths, outputPaths, storeDir);
   }
 
-  @SuppressWarnings("deprecation")
   public static CompactionDescriptor toCompactionDescriptor(HRegionInfo info, byte[] regionName,
       byte[] family, List<Path> inputPaths, List<Path> outputPaths, Path storeDir) {
     // compaction descriptor contains relative paths.
@@ -3663,4 +3674,28 @@ public final class ProtobufUtil {
     return new RegionLoadStats(stats.getMemstoreLoad(), stats.getHeapOccupancy(),
         stats.getCompactionPressure());
   }
-}
+
+  /**
+   * @param msg
+   * @return A String version of the passed in <code>msg</code>
+   */
+  public static String toText(Message msg) {
+    return TextFormat.shortDebugString(msg);
+  }
+
+  public static byte [] toBytes(ByteString bs) {
+    return bs.toByteArray();
+  }
+
+  /**
+   * Contain ServiceException inside here. Take a callable that is doing our pb rpc and run it.
+   * @throws IOException
+   */
+  public static <T> T call(Callable<T> callable) throws IOException {
+    try {
+      return callable.call();
+    } catch (Exception e) {
+      throw ProtobufUtil.handleRemoteException(e);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
index 0aa9704..5959078 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
@@ -190,7 +190,7 @@ public class TestAsyncProcess {
 
     @Override
     protected RpcRetryingCaller<MultiResponse> createCaller(
-        PayloadCarryingServerCallable callable) {
+        CancellableRegionServerCallable callable) {
       callsCt.incrementAndGet();
       MultiServerCallable callable1 = (MultiServerCallable) callable;
       final MultiResponse mr = createMultiResponse(
@@ -253,7 +253,7 @@ public class TestAsyncProcess {
 
     @Override
     protected RpcRetryingCaller<MultiResponse> createCaller(
-      PayloadCarryingServerCallable callable) {
+      CancellableRegionServerCallable callable) {
       callsCt.incrementAndGet();
       return new CallerWithFailure(ioe);
     }
@@ -290,7 +290,7 @@ public class TestAsyncProcess {
 
     @Override
     protected RpcRetryingCaller<MultiResponse> createCaller(
-        PayloadCarryingServerCallable payloadCallable) {
+        CancellableRegionServerCallable payloadCallable) {
       MultiServerCallable<Row> callable = (MultiServerCallable) payloadCallable;
       final MultiResponse mr = createMultiResponse(
           callable.getMulti(), nbMultiResponse, nbActions, new ResponseGenerator() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientScanner.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientScanner.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientScanner.java
index f083001..fd2a393 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientScanner.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientScanner.java
@@ -65,7 +65,6 @@ public class TestClientScanner {
   RpcControllerFactory controllerFactory;
 
   @Before
-  @SuppressWarnings("deprecation")
   public void setup() throws IOException {
     clusterConn = Mockito.mock(ClusterConnection.class);
     rpcFactory = Mockito.mock(RpcRetryingCallerFactory.class);

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-common/src/main/java/org/apache/hadoop/hbase/HBaseIOException.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HBaseIOException.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HBaseIOException.java
index 9c3367e..edcbdc5 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HBaseIOException.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HBaseIOException.java
@@ -45,4 +45,5 @@ public class HBaseIOException extends IOException {
 
   public HBaseIOException(Throwable cause) {
       super(cause);
-  }}
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ExceptionUtil.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ExceptionUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ExceptionUtil.java
index 688b51a..7e6c5d6 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ExceptionUtil.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ExceptionUtil.java
@@ -73,4 +73,4 @@ public class ExceptionUtil {
 
   private ExceptionUtil() {
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
index 5b2aab1..4b27924 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
@@ -83,9 +83,9 @@ import org.apache.hadoop.hbase.codec.Codec;
 import org.apache.hadoop.hbase.conf.ConfigurationObserver;
 import org.apache.hadoop.hbase.exceptions.RegionMovedException;
 import org.apache.hadoop.hbase.io.ByteBufferInputStream;
+import org.apache.hadoop.hbase.io.ByteBufferListOutputStream;
 import org.apache.hadoop.hbase.io.ByteBufferOutputStream;
 import org.apache.hadoop.hbase.io.ByteBufferPool;
-import org.apache.hadoop.hbase.io.ByteBufferListOutputStream;
 import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler;
 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
index 09dedec..a34dc0a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
@@ -20,11 +20,6 @@ package org.apache.hadoop.hbase.mapreduce;
 
 import static java.lang.String.format;
 
-import com.google.common.collect.HashMultimap;
-import com.google.common.collect.Multimap;
-import com.google.common.collect.Multimaps;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InterruptedIOException;
@@ -87,12 +82,12 @@ import org.apache.hadoop.hbase.io.hfile.HFileContext;
 import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
 import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder;
 import org.apache.hadoop.hbase.io.hfile.HFileScanner;
+import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.regionserver.BloomType;
 import org.apache.hadoop.hbase.regionserver.HStore;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
 import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
 import org.apache.hadoop.hbase.security.UserProvider;
-import org.apache.hadoop.hbase.security.access.SecureBulkLoadEndpoint;
 import org.apache.hadoop.hbase.security.token.FsDelegationToken;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSHDFSUtils;
@@ -100,9 +95,13 @@ import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Multimap;
+import com.google.common.collect.Multimaps;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
 /**
  * Tool to load the output of HFileOutputFormat into an existing table.
- * @see #usage()
  */
 @InterfaceAudience.Public
 @InterfaceStability.Stable
@@ -130,11 +129,13 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
   private String bulkToken;
   private UserProvider userProvider;
   private int nrThreads;
+  private RpcControllerFactory rpcControllerFactory;
 
   private LoadIncrementalHFiles() {}
 
   public LoadIncrementalHFiles(Configuration conf) throws Exception {
     super(conf);
+    this.rpcControllerFactory = new RpcControllerFactory(conf);
     initialize();
   }
 
@@ -322,7 +323,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
     // LQI queue does not need to be threadsafe -- all operations on this queue
     // happen in this thread
     Deque<LoadQueueItem> queue = new LinkedList<>();
-    SecureBulkLoadClient secureClient =  new SecureBulkLoadClient(table);
+    SecureBulkLoadClient secureClient =  new SecureBulkLoadClient(table.getConfiguration(), table);
 
     try {
       /*
@@ -473,9 +474,11 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
 
   /**
    * Used by the replication sink to load the hfiles from the source cluster. It does the following,
-   * 1. {@link LoadIncrementalHFiles#groupOrSplitPhase(Table, ExecutorService, Deque, Pair)} 2.
-   * {@link
-   * LoadIncrementalHFiles#bulkLoadPhase(Table, Connection, ExecutorService, Deque, Multimap)}
+   * <ol>
+   * <li>LoadIncrementalHFiles#groupOrSplitPhase(Table, ExecutorService, Deque, Pair)}</li>
+   * <li>LoadIncrementalHFiles#bulkLoadPhase(Table, Connection, ExecutorService, Deque, Multimap)
+   * </li>
+   * </ol>
    * @param table Table to which these hfiles should be loaded to
    * @param conn Connection to use
    * @param queue {@link LoadQueueItem} has hfiles yet to be loaded
@@ -776,27 +779,23 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
   protected List<LoadQueueItem> tryAtomicRegionLoad(final Connection conn,
       final TableName tableName, final byte[] first, final Collection<LoadQueueItem> lqis)
   throws IOException {
-    final List<Pair<byte[], String>> famPaths =
-      new ArrayList<>(lqis.size());
+    final List<Pair<byte[], String>> famPaths = new ArrayList<>(lqis.size());
     for (LoadQueueItem lqi : lqis) {
       famPaths.add(Pair.newPair(lqi.family, lqi.hfilePath.toString()));
     }
-
-    final RegionServerCallable<Boolean> svrCallable =
-        new RegionServerCallable<Boolean>(conn, tableName, first) {
+    final RegionServerCallable<Boolean> svrCallable = new RegionServerCallable<Boolean>(conn,
+        rpcControllerFactory, tableName, first) {
       @Override
-      public Boolean call(int callTimeout) throws Exception {
+      protected Boolean rpcCall() throws Exception {
         SecureBulkLoadClient secureClient = null;
         boolean success = false;
-
         try {
           LOG.debug("Going to connect to server " + getLocation() + " for row "
               + Bytes.toStringBinary(getRow()) + " with hfile group " + famPaths);
           byte[] regionName = getLocation().getRegionInfo().getRegionName();
           try (Table table = conn.getTable(getTableName())) {
-            secureClient = new SecureBulkLoadClient(table);
-            success =
-                secureClient.secureBulkLoadHFiles(getStub(), famPaths, regionName,
+            secureClient = new SecureBulkLoadClient(getConf(), table);
+            success = secureClient.secureBulkLoadHFiles(getStub(), famPaths, regionName,
                   assignSeqIds, fsDelegationToken.getUserToken(), bulkToken);
           }
           return success;
@@ -1078,7 +1077,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
 
   /**
    * Called from replication sink, where it manages bulkToken(staging directory) by itself. This is
-   * used only when {@link SecureBulkLoadEndpoint} is configured in hbase.coprocessor.region.classes
+   * used only when SecureBulkLoadEndpoint is configured in hbase.coprocessor.region.classes
    * property. This directory is used as a temporary directory where all files are initially
    * copied/moved from user given directory, set all the required file permissions and then from
    * their it is finally loaded into a table. This should be set only when, one would like to manage
@@ -1088,5 +1087,4 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
   public void setBulkToken(String stagingDir) {
     this.bulkToken = stagingDir;
   }
-
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ExpiredMobFileCleanerChore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ExpiredMobFileCleanerChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ExpiredMobFileCleanerChore.java
index a21edcc..3261bd6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ExpiredMobFileCleanerChore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ExpiredMobFileCleanerChore.java
@@ -35,8 +35,6 @@ import org.apache.hadoop.hbase.mob.ExpiredMobFileCleaner;
 import org.apache.hadoop.hbase.mob.MobConstants;
 import org.apache.hadoop.hbase.mob.MobUtils;
 
-import com.google.protobuf.ServiceException;
-
 /**
  * The Class ExpiredMobFileCleanerChore for running cleaner regularly to remove the expired
  * mob files.
@@ -86,10 +84,6 @@ public class ExpiredMobFileCleanerChore extends ScheduledChore {
             } catch (LockTimeoutException e) {
               LOG.info("Fail to acquire the lock because of timeout, maybe a"
                 + " MobCompactor is running", e);
-            } catch (ServiceException e) {
-              LOG.error(
-                "Fail to clean the expired mob files for the column " + hcd.getNameAsString()
-                  + " in the table " + htd.getNameAsString(), e);
             } catch (IOException e) {
               LOG.error(
                 "Fail to clean the expired mob files for the column " + hcd.getNameAsString()

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index ad1a3ca..326aa00 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -18,14 +18,6 @@
  */
 package org.apache.hadoop.hbase.master;
 
-import com.google.protobuf.ByteString;
-import com.google.protobuf.Descriptors;
-import com.google.protobuf.Message;
-import com.google.protobuf.RpcCallback;
-import com.google.protobuf.RpcController;
-import com.google.protobuf.Service;
-import com.google.protobuf.ServiceException;
-
 import java.io.IOException;
 import java.net.InetAddress;
 import java.util.ArrayList;
@@ -92,7 +84,6 @@ import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.Repor
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse;
 import org.apache.hadoop.hbase.regionserver.RSRpcServices;
-import org.apache.hadoop.hbase.security.AccessDeniedException;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.access.AccessController;
 import org.apache.hadoop.hbase.security.visibility.VisibilityController;
@@ -103,6 +94,14 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.zookeeper.KeeperException;
 
+import com.google.protobuf.ByteString;
+import com.google.protobuf.Descriptors;
+import com.google.protobuf.Message;
+import com.google.protobuf.RpcCallback;
+import com.google.protobuf.RpcController;
+import com.google.protobuf.Service;
+import com.google.protobuf.ServiceException;
+
 /**
  * Implements the master RPC services.
  */

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
index 531883a..d7ba4f3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
@@ -61,9 +61,9 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionRequest;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionResponse;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ServerInfo;
-import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
 import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds;
 import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.StoreSequenceId;
+import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
@@ -454,8 +454,7 @@ public class ServerManager {
   /**
    * Adds the onlineServers list. onlineServers should be locked.
    * @param serverName The remote servers name.
-   * @param sl
-   * @return Server load from the removed server, if any.
+   * @param s
    */
   @VisibleForTesting
   void recordNewServerWithLock(final ServerName serverName, final ServerLoad sl) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java
index 1499788..96ea036 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java
@@ -188,7 +188,6 @@ public class TableStateManager {
     return MetaTableAccessor.getTableState(master.getConnection(), tableName);
   }
 
-  @SuppressWarnings("deprecation")
   public void start() throws IOException {
     TableDescriptors tableDescriptors = master.getTableDescriptors();
     Connection connection = master.getConnection();
@@ -220,4 +219,4 @@ public class TableStateManager {
       }
     }
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/ExpiredMobFileCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/ExpiredMobFileCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/ExpiredMobFileCleaner.java
index 3c965cb..d4a54bb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/ExpiredMobFileCleaner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/ExpiredMobFileCleaner.java
@@ -41,8 +41,6 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 
-import com.google.protobuf.ServiceException;
-
 /**
  * The cleaner to delete the expired MOB files.
  */
@@ -60,11 +58,8 @@ public class ExpiredMobFileCleaner extends Configured implements Tool {
    * directory.
    * @param tableName The current table name.
    * @param family The current family.
-   * @throws ServiceException
-   * @throws IOException
    */
-  public void cleanExpiredMobFiles(String tableName, HColumnDescriptor family)
-      throws ServiceException, IOException {
+  public void cleanExpiredMobFiles(String tableName, HColumnDescriptor family) throws IOException {
     Configuration conf = getConf();
     TableName tn = TableName.valueOf(tableName);
     FileSystem fs = FileSystem.get(conf);
@@ -99,7 +94,7 @@ public class ExpiredMobFileCleaner extends Configured implements Tool {
     String tableName = args[0];
     String familyName = args[1];
     TableName tn = TableName.valueOf(tableName);
-    HBaseAdmin.checkHBaseAvailable(getConf());
+    HBaseAdmin.available(getConf());
     Connection connection = ConnectionFactory.createConnection(getConf());
     Admin admin = connection.getAdmin();
     try {
@@ -127,5 +122,4 @@ public class ExpiredMobFileCleaner extends Configured implements Tool {
       }
     }
   }
-
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/mapreduce/Sweeper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/mapreduce/Sweeper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/mapreduce/Sweeper.java
index 8547c8c..c27e8ae 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/mapreduce/Sweeper.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/mapreduce/Sweeper.java
@@ -38,8 +38,6 @@ import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.zookeeper.KeeperException;
 
-import com.google.protobuf.ServiceException;
-
 /**
  * The sweep tool. It deletes the mob files that are not used and merges the small mob files to
  * bigger ones. Each run of this sweep tool only handles one column family. The runs on
@@ -64,10 +62,10 @@ public class Sweeper extends Configured implements Tool {
    * @throws ServiceException
    */
   int sweepFamily(String tableName, String familyName) throws IOException, InterruptedException,
-      ClassNotFoundException, KeeperException, ServiceException {
+      ClassNotFoundException, KeeperException {
     Configuration conf = getConf();
     // make sure the target HBase exists.
-    HBaseAdmin.checkHBaseAvailable(conf);
+    HBaseAdmin.available(conf);
     Connection connection = ConnectionFactory.createConnection(getConf());
     Admin admin = connection.getAdmin();
     try {

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index 0b4ae75..89bfbf3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -18,17 +18,6 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Maps;
-import com.google.protobuf.BlockingRpcChannel;
-import com.google.protobuf.Descriptors;
-import com.google.protobuf.Message;
-import com.google.protobuf.RpcCallback;
-import com.google.protobuf.RpcController;
-import com.google.protobuf.Service;
-import com.google.protobuf.ServiceException;
-
 import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.lang.Thread.UncaughtExceptionHandler;
@@ -106,7 +95,6 @@ import org.apache.hadoop.hbase.exceptions.UnknownProtocolException;
 import org.apache.hadoop.hbase.executor.ExecutorService;
 import org.apache.hadoop.hbase.executor.ExecutorType;
 import org.apache.hadoop.hbase.fs.HFileSystem;
-import org.apache.hadoop.hbase.http.HttpServer;
 import org.apache.hadoop.hbase.http.InfoServer;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.HFile;
@@ -198,6 +186,17 @@ import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.KeeperException.NoNodeException;
 import org.apache.zookeeper.data.Stat;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Maps;
+import com.google.protobuf.BlockingRpcChannel;
+import com.google.protobuf.Descriptors;
+import com.google.protobuf.Message;
+import com.google.protobuf.RpcCallback;
+import com.google.protobuf.RpcController;
+import com.google.protobuf.Service;
+import com.google.protobuf.ServiceException;
+
 import sun.misc.Signal;
 import sun.misc.SignalHandler;
 
@@ -206,7 +205,7 @@ import sun.misc.SignalHandler;
  * the HMaster. There are many HRegionServers in a single HBase deployment.
  */
 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
-@SuppressWarnings("deprecation")
+@SuppressWarnings({ "deprecation", "restriction" })
 public class HRegionServer extends HasThread implements
     RegionServerServices, LastSequenceId, ConfigurationObserver {
 
@@ -818,9 +817,8 @@ public class HRegionServer extends HasThread implements
     // when ready.
     blockAndCheckIfStopped(this.clusterStatusTracker);
 
-    if (this.initLatch != null) {
-      this.initLatch.await(20, TimeUnit.SECONDS);
-    }
+    doLatch(this.initLatch);
+
     // Retrieve clusterId
     // Since cluster status is now up
     // ID should have already been set by HMaster
@@ -855,6 +853,16 @@ public class HRegionServer extends HasThread implements
     this.recoveringRegionWatcher = new RecoveringRegionWatcher(this.zooKeeper, this);
   }
 
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="RV_RETURN_VALUE_IGNORED",
+      justification="We don't care about the return")
+  private void doLatch(final CountDownLatch latch) throws InterruptedException {
+    if (latch != null) {
+      // Result is ignored intentionally but if I remove the below, findbugs complains (the
+      // above justification on this method doesn't seem to suppress it).
+      boolean result = latch.await(20, TimeUnit.SECONDS);
+    }
+  }
+
   /**
    * Utilty method to wait indefinitely on a znode availability while checking
    * if the region server is shut down

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index 681b1dc..3859d18 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -87,7 +87,6 @@ import org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface;
 import org.apache.hadoop.hbase.ipc.RpcServerInterface;
 import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
 import org.apache.hadoop.hbase.ipc.ServerRpcController;
-import org.apache.hadoop.hbase.ipc.TimeLimitedRpcController;
 import org.apache.hadoop.hbase.master.MasterRpcServices;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.RequestConverter;
@@ -1381,8 +1380,9 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
         family = request.getFamily().toByteArray();
         store = region.getStore(family);
         if (store == null) {
-          throw new ServiceException(new IOException("column family " + Bytes.toString(family)
-            + " does not exist in region " + region.getRegionInfo().getRegionNameAsString()));
+          throw new ServiceException(new DoNotRetryIOException("column family " +
+              Bytes.toString(family) + " does not exist in region " +
+              region.getRegionInfo().getRegionNameAsString()));
         }
       }
       if (request.hasMajor()) {
@@ -2767,12 +2767,16 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
                     timeLimitDelta =
                         scannerLeaseTimeoutPeriod > 0 ? scannerLeaseTimeoutPeriod : rpcTimeout;
                   }
-                  if (controller instanceof TimeLimitedRpcController) {
-                    TimeLimitedRpcController timeLimitedRpcController =
-                        (TimeLimitedRpcController)controller;
-                    if (timeLimitedRpcController.getCallTimeout() > 0) {
-                      timeLimitDelta = Math.min(timeLimitDelta,
-                          timeLimitedRpcController.getCallTimeout());
+                  if (controller != null) {
+                    if (controller instanceof PayloadCarryingRpcController) {
+                      PayloadCarryingRpcController pRpcController =
+                          (PayloadCarryingRpcController)controller;
+                      if (pRpcController.getCallTimeout() > 0) {
+                        timeLimitDelta = Math.min(timeLimitDelta, pRpcController.getCallTimeout());
+                      }
+                    } else {
+                      throw new UnsupportedOperationException("We only do " +
+                        "PayloadCarryingRpcControllers! FIX IF A PROBLEM: " + controller);
                     }
                   }
                   // Use half of whichever timeout value was more restrictive... But don't allow

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEditsReplaySink.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEditsReplaySink.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEditsReplaySink.java
index 3eb85bd..004581d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEditsReplaySink.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEditsReplaySink.java
@@ -18,8 +18,6 @@
  */
 package org.apache.hadoop.hbase.regionserver.wal;
 
-import com.google.protobuf.ServiceException;
-
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
@@ -42,9 +40,7 @@ import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.RegionServerCallable;
 import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
-import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
-import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.ReplicationProtbufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
@@ -61,10 +57,8 @@ import org.apache.hadoop.hbase.wal.WAL.Entry;
  */
 @InterfaceAudience.Private
 public class WALEditsReplaySink {
-
   private static final Log LOG = LogFactory.getLog(WALEditsReplaySink.class);
   private static final int MAX_BATCH_SIZE = 1024;
-
   private final Configuration conf;
   private final ClusterConnection conn;
   private final TableName tableName;
@@ -166,8 +160,8 @@ public class WALEditsReplaySink {
     try {
       RpcRetryingCallerFactory factory = RpcRetryingCallerFactory.instantiate(conf, null);
       ReplayServerCallable<ReplicateWALEntryResponse> callable =
-          new ReplayServerCallable<ReplicateWALEntryResponse>(this.conn, this.tableName, regionLoc,
-              regionInfo, entries);
+          new ReplayServerCallable<ReplicateWALEntryResponse>(this.conn, this.rpcControllerFactory,
+              this.tableName, regionLoc, entries);
       factory.<ReplicateWALEntryResponse> newCaller().callWithRetries(callable, this.replayTimeout);
     } catch (IOException ie) {
       if (skipErrors) {
@@ -184,31 +178,18 @@ public class WALEditsReplaySink {
    * @param <R>
    */
   class ReplayServerCallable<R> extends RegionServerCallable<ReplicateWALEntryResponse> {
-    private HRegionInfo regionInfo;
     private List<Entry> entries;
 
-    ReplayServerCallable(final Connection connection, final TableName tableName,
-        final HRegionLocation regionLoc, final HRegionInfo regionInfo,
-        final List<Entry> entries) {
-      super(connection, tableName, null);
+    ReplayServerCallable(final Connection connection, RpcControllerFactory rpcControllerFactory,
+        final TableName tableName, final HRegionLocation regionLoc, final List<Entry> entries) {
+      super(connection, rpcControllerFactory, tableName, null);
       this.entries = entries;
-      this.regionInfo = regionInfo;
       setLocation(regionLoc);
     }
 
     @Override
-    public ReplicateWALEntryResponse call(int callTimeout) throws IOException {
-      try {
-        replayToServer(this.regionInfo, this.entries);
-      } catch (ServiceException se) {
-        throw ProtobufUtil.getRemoteException(se);
-      }
-      return null;
-    }
-
-    private void replayToServer(HRegionInfo regionInfo, List<Entry> entries)
-        throws IOException, ServiceException {
-      if (entries.isEmpty()) return;
+    protected ReplicateWALEntryResponse rpcCall() throws Exception {
+      if (entries.isEmpty()) return null;
 
       Entry[] entriesArray = new Entry[entries.size()];
       entriesArray = entries.toArray(entriesArray);
@@ -216,12 +197,8 @@ public class WALEditsReplaySink {
 
       Pair<AdminProtos.ReplicateWALEntryRequest, CellScanner> p =
           ReplicationProtbufUtil.buildReplicateWALEntryRequest(entriesArray);
-      PayloadCarryingRpcController controller = rpcControllerFactory.newController(p.getSecond());
-      try {
-        remoteSvr.replay(controller, p.getFirst());
-      } catch (ServiceException se) {
-        throw ProtobufUtil.getRemoteException(se);
-      }
+      setRpcControllerCellScanner(p.getSecond());
+      return remoteSvr.replay(getRpcController(), p.getFirst());
     }
 
     @Override
@@ -245,4 +222,4 @@ public class WALEditsReplaySink {
       }
     }
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java
index b0fd176..c756294 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java
@@ -34,7 +34,6 @@ import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.CellScanner;
@@ -46,27 +45,21 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.RegionLocations;
+import org.apache.hadoop.hbase.TableDescriptors;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.hadoop.hbase.TableDescriptors;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.RegionAdminServiceCallable;
-import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
 import org.apache.hadoop.hbase.client.RetryingCallable;
 import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
 import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
-import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.ReplicationProtbufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryResponse;
-import org.apache.hadoop.hbase.wal.WAL.Entry;
-import org.apache.hadoop.hbase.wal.WALSplitter.EntryBuffers;
-import org.apache.hadoop.hbase.wal.WALSplitter.OutputSink;
-import org.apache.hadoop.hbase.wal.WALSplitter.PipelineController;
-import org.apache.hadoop.hbase.wal.WALSplitter.RegionEntryBuffer;
-import org.apache.hadoop.hbase.wal.WALSplitter.SinkWriter;
 import org.apache.hadoop.hbase.replication.BaseWALEntryFilter;
 import org.apache.hadoop.hbase.replication.ChainWALEntryFilter;
 import org.apache.hadoop.hbase.replication.HBaseReplicationEndpoint;
@@ -74,12 +67,17 @@ import org.apache.hadoop.hbase.replication.WALEntryFilter;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.Threads;
+import org.apache.hadoop.hbase.wal.WAL.Entry;
+import org.apache.hadoop.hbase.wal.WALSplitter.EntryBuffers;
+import org.apache.hadoop.hbase.wal.WALSplitter.OutputSink;
+import org.apache.hadoop.hbase.wal.WALSplitter.PipelineController;
+import org.apache.hadoop.hbase.wal.WALSplitter.RegionEntryBuffer;
+import org.apache.hadoop.hbase.wal.WALSplitter.SinkWriter;
 import org.apache.hadoop.util.StringUtils;
 
 import com.google.common.cache.Cache;
 import com.google.common.cache.CacheBuilder;
 import com.google.common.collect.Lists;
-import com.google.protobuf.ServiceException;
 
 /**
  * A {@link org.apache.hadoop.hbase.replication.ReplicationEndpoint} endpoint
@@ -611,9 +609,8 @@ public class RegionReplicaReplicationEndpoint extends HBaseReplicationEndpoint {
    * Calls replay on the passed edits for the given set of entries belonging to the region. It skips
    * the entry if the region boundaries have changed or the region is gone.
    */
-  static class RegionReplicaReplayCallable
-    extends RegionAdminServiceCallable<ReplicateWALEntryResponse> {
-
+  static class RegionReplicaReplayCallable extends
+      RegionAdminServiceCallable<ReplicateWALEntryResponse> {
     private final List<Entry> entries;
     private final byte[] initialEncodedRegionName;
     private final AtomicLong skippedEntries;
@@ -628,38 +625,25 @@ public class RegionReplicaReplicationEndpoint extends HBaseReplicationEndpoint {
       this.initialEncodedRegionName = regionInfo.getEncodedNameAsBytes();
     }
 
-    @Override
-    public ReplicateWALEntryResponse call(int timeout) throws IOException {
-      return replayToServer(this.entries, timeout);
-    }
-
-    private ReplicateWALEntryResponse replayToServer(List<Entry> entries, int timeout)
-        throws IOException {
-      // check whether we should still replay this entry. If the regions are changed, or the
+    public ReplicateWALEntryResponse call(PayloadCarryingRpcController controller) throws Exception {
+      // Check whether we should still replay this entry. If the regions are changed, or the
       // entry is not coming form the primary region, filter it out because we do not need it.
       // Regions can change because of (1) region split (2) region merge (3) table recreated
       boolean skip = false;
-
       if (!Bytes.equals(location.getRegionInfo().getEncodedNameAsBytes(),
-        initialEncodedRegionName)) {
+          initialEncodedRegionName)) {
         skip = true;
       }
-      if (!entries.isEmpty() && !skip) {
-        Entry[] entriesArray = new Entry[entries.size()];
-        entriesArray = entries.toArray(entriesArray);
+      if (!this.entries.isEmpty() && !skip) {
+        Entry[] entriesArray = new Entry[this.entries.size()];
+        entriesArray = this.entries.toArray(entriesArray);
 
         // set the region name for the target region replica
         Pair<AdminProtos.ReplicateWALEntryRequest, CellScanner> p =
             ReplicationProtbufUtil.buildReplicateWALEntryRequest(entriesArray, location
                 .getRegionInfo().getEncodedNameAsBytes(), null, null, null);
-        try {
-          PayloadCarryingRpcController controller = rpcControllerFactory.newController(p.getSecond());
-          controller.setCallTimeout(timeout);
-          controller.setPriority(tableName);
-          return stub.replay(controller, p.getFirst());
-        } catch (ServiceException se) {
-          throw ProtobufUtil.getRemoteException(se);
-        }
+        controller.setCellScanner(p.getSecond());
+        return stub.replay(controller, p.getFirst());
       }
 
       if (skip) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
index 2e7cf7f..bbf858d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
@@ -1145,8 +1145,11 @@ public final class Canary implements Tool {
     }
     List<RegionTask> tasks = new ArrayList<RegionTask>();
     try {
-      for (HRegionInfo region : admin.getTableRegions(tableDesc.getTableName())) {        
-        tasks.add(new RegionTask(admin.getConnection(), region, sink, taskType, rawScanEnabled));
+      List<HRegionInfo> hris = admin.getTableRegions(tableDesc.getTableName());
+      if (hris != null) {
+        for (HRegionInfo region : hris) {
+          tasks.add(new RegionTask(admin.getConnection(), region, sink, taskType, rawScanEnabled));
+        }
       }
     } finally {
       table.close();

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-server/src/main/java/org/apache/hadoop/hbase/util/Merge.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/Merge.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/Merge.java
index d708edc..3c81cfe 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/Merge.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/Merge.java
@@ -23,19 +23,18 @@ import java.io.IOException;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.MetaTableAccessor;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.MasterNotRunningException;
+import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
@@ -80,13 +79,11 @@ public class Merge extends Configured implements Tool {
     // Verify HBase is down
     LOG.info("Verifying that HBase is not running...");
     try {
-      HBaseAdmin.checkHBaseAvailable(getConf());
+      HBaseAdmin.available(getConf());
       LOG.fatal("HBase cluster must be off-line, and is not. Aborting.");
       return -1;
     } catch (ZooKeeperConnectionException zkce) {
       // If no zk, presume no master.
-    } catch (MasterNotRunningException e) {
-      // Expected. Ignore.
     }
 
     // Initialize MetaUtils and and get the root of the HBase installation

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java
index d778fa9..2dca6b1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java
@@ -60,7 +60,6 @@ public class TestNamespace {
   private static ZKNamespaceManager zkNamespaceManager;
   private String prefix = "TestNamespace";
 
-
   @BeforeClass
   public static void setUp() throws Exception {
     TEST_UTIL = new HBaseTestingUtility();
@@ -301,7 +300,8 @@ public class TestNamespace {
     runWithExpectedException(new Callable<Void>() {
       @Override
       public Void call() throws Exception {
-        HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("non_existing_namespace", "table1"));
+        HTableDescriptor htd =
+            new HTableDescriptor(TableName.valueOf("non_existing_namespace", "table1"));
         htd.addFamily(new HColumnDescriptor("family1"));
         admin.createTable(htd);
         return null;
@@ -387,5 +387,4 @@ public class TestNamespace {
     }
     fail("Should have thrown exception " + exceptionClass);
   }
-
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
index d088fc4..3203636 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
@@ -37,7 +37,6 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.MasterNotRunningException;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.NotServingRegionException;
 import org.apache.hadoop.hbase.ProcedureInfo;
@@ -59,7 +58,6 @@ import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
-import org.apache.hadoop.hbase.wal.FSHLogProvider;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -67,8 +65,6 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import com.google.protobuf.ServiceException;
-
 
 /**
  * Class to test HBaseAdmin.
@@ -335,7 +331,8 @@ public class TestAdmin2 {
 
   @Test (timeout=300000)
   public void testCloseRegionIfInvalidRegionNameIsPassed() throws Exception {
-    byte[] TABLENAME = Bytes.toBytes("TestHBACloseRegion1");
+    final String name = "TestHBACloseRegion1";
+    byte[] TABLENAME = Bytes.toBytes(name);
     createTableWithDefaultConf(TABLENAME);
 
     HRegionInfo info = null;
@@ -343,7 +340,7 @@ public class TestAdmin2 {
     List<HRegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(rs.getRSRpcServices());
     for (HRegionInfo regionInfo : onlineRegions) {
       if (!regionInfo.isMetaTable()) {
-        if (regionInfo.getRegionNameAsString().contains("TestHBACloseRegion1")) {
+        if (regionInfo.getRegionNameAsString().contains(name)) {
           info = regionInfo;
           try {
             admin.closeRegionWithEncodedRegionName("sample", rs.getServerName()
@@ -643,11 +640,9 @@ public class TestAdmin2 {
 
     long start = System.currentTimeMillis();
     try {
-      HBaseAdmin.checkHBaseAvailable(conf);
+      HBaseAdmin.available(conf);
       assertTrue(false);
-    } catch (MasterNotRunningException ignored) {
     } catch (ZooKeeperConnectionException ignored) {
-    } catch (ServiceException ignored) {
     } catch (IOException ignored) {
     }
     long end = System.currentTimeMillis();

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java
index 679d9c9..f49c558 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java
@@ -28,13 +28,10 @@ import java.net.UnknownHostException;
 import java.util.Random;
 import java.util.concurrent.atomic.AtomicInteger;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.MasterNotRunningException;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
 import org.apache.hadoop.hbase.ipc.AbstractRpcClient;
@@ -56,7 +53,6 @@ import com.google.protobuf.ServiceException;
 
 @Category({MediumTests.class, ClientTests.class})
 public class TestClientTimeouts {
-  private static final Log LOG = LogFactory.getLog(TestClientTimeouts.class);
   private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
   protected static int SLAVES = 1;
 
@@ -87,7 +83,6 @@ public class TestClientTimeouts {
    */
   @Test
   public void testAdminTimeout() throws Exception {
-    Connection lastConnection = null;
     boolean lastFailed = false;
     int initialInvocations = RandomTimeoutBlockingRpcChannel.invokations.get();
     RandomTimeoutRpcClient rpcClient = (RandomTimeoutRpcClient) RpcClientFactory
@@ -105,7 +100,7 @@ public class TestClientTimeouts {
           connection = ConnectionFactory.createConnection(conf);
           admin = connection.getAdmin();
           // run some admin commands
-          HBaseAdmin.checkHBaseAvailable(conf);
+          HBaseAdmin.available(conf);
           admin.setBalancerRunning(false, false);
         } catch (ZooKeeperConnectionException ex) {
           // Since we are randomly throwing SocketTimeoutExceptions, it is possible to get

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/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 bfd16a7..bda80de 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
@@ -58,6 +58,7 @@ import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.exceptions.RegionMovedException;
 import org.apache.hadoop.hbase.filter.Filter;
 import org.apache.hadoop.hbase.filter.FilterBase;
+import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
 import org.apache.hadoop.hbase.ipc.RpcClient;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.master.HMaster;
@@ -84,6 +85,8 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
+import com.google.common.collect.Lists;
+import com.google.protobuf.RpcController;
 
 /**
  * This class is for testing HBaseConnectionManager features
@@ -104,8 +107,6 @@ public class TestHCM {
       TableName.valueOf("test2");
   private static final TableName TABLE_NAME3 =
       TableName.valueOf("test3");
-  private static final TableName TABLE_NAME4 =
-      TableName.valueOf("test4");
   private static final byte[] FAM_NAM = Bytes.toBytes("f");
   private static final byte[] ROW = Bytes.toBytes("bbb");
   private static final byte[] ROW_X = Bytes.toBytes("xxx");
@@ -525,10 +526,12 @@ public class TestHCM {
     long pauseTime;
     long baseTime = 100;
     TableName tableName = TableName.valueOf("HCM-testCallableSleep");
-    Table table = TEST_UTIL.createTable(tableName, FAM_NAM);
+    TEST_UTIL.createTable(tableName, FAM_NAM);
     RegionServerCallable<Object> regionServerCallable = new RegionServerCallable<Object>(
-        TEST_UTIL.getConnection(), tableName, ROW) {
-      public Object call(int timeout) throws IOException {
+        TEST_UTIL.getConnection(), new RpcControllerFactory(TEST_UTIL.getConfiguration()),
+        tableName, ROW) {
+      @Override
+      protected Object rpcCall() throws Exception {
         return null;
       }
     };
@@ -542,9 +545,10 @@ public class TestHCM {
 
     RegionAdminServiceCallable<Object> regionAdminServiceCallable =
         new RegionAdminServiceCallable<Object>(
-        (ClusterConnection) TEST_UTIL.getConnection(), new RpcControllerFactory(
-            TEST_UTIL.getConfiguration()), tableName, ROW) {
-      public Object call(int timeout) throws IOException {
+        (ClusterConnection) TEST_UTIL.getConnection(),
+          new RpcControllerFactory(TEST_UTIL.getConfiguration()), tableName, ROW) {
+      @Override
+      public Object call(PayloadCarryingRpcController controller) throws Exception {
         return null;
       }
     };
@@ -556,16 +560,21 @@ public class TestHCM {
       assertTrue(pauseTime <= (baseTime * HConstants.RETRY_BACKOFF[i] * 1.01f));
     }
 
-    MasterCallable masterCallable = new MasterCallable(TEST_UTIL.getConnection()) {
-      public Object call(int timeout) throws IOException {
+    MasterCallable<Object> masterCallable = new MasterCallable<Object>(TEST_UTIL.getConnection(),
+        new RpcControllerFactory(TEST_UTIL.getConfiguration())) {
+      @Override
+      protected Object rpcCall() throws Exception {
         return null;
       }
     };
-
-    for (int i = 0; i < HConstants.RETRY_BACKOFF.length; i++) {
-      pauseTime = masterCallable.sleep(baseTime, i);
-      assertTrue(pauseTime >= (baseTime * HConstants.RETRY_BACKOFF[i]));
-      assertTrue(pauseTime <= (baseTime * HConstants.RETRY_BACKOFF[i] * 1.01f));
+    try {
+      for (int i = 0; i < HConstants.RETRY_BACKOFF.length; i++) {
+        pauseTime = masterCallable.sleep(baseTime, i);
+        assertTrue(pauseTime >= (baseTime * HConstants.RETRY_BACKOFF[i]));
+        assertTrue(pauseTime <= (baseTime * HConstants.RETRY_BACKOFF[i] * 1.01f));
+      }
+    } finally {
+      masterCallable.close();
     }
   }
 
@@ -1267,7 +1276,6 @@ public class TestHCM {
     ManualEnvironmentEdge timeMachine = new ManualEnvironmentEdge();
     EnvironmentEdgeManager.injectEdge(timeMachine);
     try {
-      long timeBase = timeMachine.currentTime();
       long largeAmountOfTime = ANY_PAUSE * 1000;
       ConnectionImplementation.ServerErrorTracker tracker =
           new ConnectionImplementation.ServerErrorTracker(largeAmountOfTime, 100);

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
index 354f0a8..9b4e9f7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
@@ -20,6 +20,15 @@
 
 package org.apache.hadoop.hbase.client;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -35,6 +44,7 @@ import org.apache.hadoop.hbase.client.replication.ReplicationAdmin;
 import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.regionserver.StorefileRefresherChore;
 import org.apache.hadoop.hbase.regionserver.TestHRegionServerBulkLoad;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
@@ -49,15 +59,6 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.atomic.AtomicReference;
-
 @Category({MediumTests.class, ClientTests.class})
 public class TestReplicaWithCluster {
   private static final Log LOG = LogFactory.getLog(TestReplicaWithCluster.class);
@@ -332,26 +333,27 @@ public class TestReplicaWithCluster {
 
     // bulk load HFiles
     LOG.debug("Loading test data");
-    @SuppressWarnings("deprecation")
     final ClusterConnection conn = (ClusterConnection) HTU.getAdmin().getConnection();
     table = conn.getTable(hdt.getTableName());
-    final String bulkToken = new SecureBulkLoadClient(table).prepareBulkLoad(conn);
-    RegionServerCallable<Void> callable = new RegionServerCallable<Void>(
-      conn, hdt.getTableName(), TestHRegionServerBulkLoad.rowkey(0)) {
-        @Override
-        public Void call(int timeout) throws Exception {
-          LOG.debug("Going to connect to server " + getLocation() + " for row "
+    final String bulkToken =
+        new SecureBulkLoadClient(HTU.getConfiguration(), table).prepareBulkLoad(conn);
+    RegionServerCallable<Void> callable = new RegionServerCallable<Void>(conn,
+        new RpcControllerFactory(HTU.getConfiguration()), hdt.getTableName(),
+        TestHRegionServerBulkLoad.rowkey(0)) {
+      @Override
+      protected Void rpcCall() throws Exception {
+        LOG.debug("Going to connect to server " + getLocation() + " for row "
             + Bytes.toStringBinary(getRow()));
-          SecureBulkLoadClient secureClient = null;
-          byte[] regionName = getLocation().getRegionInfo().getRegionName();
-          try (Table table = conn.getTable(getTableName())) {
-            secureClient = new SecureBulkLoadClient(table);
-            secureClient.secureBulkLoadHFiles(getStub(), famPaths, regionName,
-                  true, null, bulkToken);
-          }
-          return null;
+        SecureBulkLoadClient secureClient = null;
+        byte[] regionName = getLocation().getRegionInfo().getRegionName();
+        try (Table table = conn.getTable(getTableName())) {
+          secureClient = new SecureBulkLoadClient(HTU.getConfiguration(), table);
+          secureClient.secureBulkLoadHFiles(getStub(), famPaths, regionName,
+              true, null, bulkToken);
         }
-      };
+        return null;
+      }
+    };
     RpcRetryingCallerFactory factory = new RpcRetryingCallerFactory(HTU.getConfiguration());
     RpcRetryingCaller<Void> caller = factory.newCaller();
     caller.callWithRetries(callable, 10000);

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java
index b3cbd33..ffe3e82 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java
@@ -573,11 +573,11 @@ public class TestReplicasClient {
         Assert.assertTrue(((Result)r).isStale());
         Assert.assertTrue(((Result)r).getExists());
       }
-      Set<PayloadCarryingServerCallable> set =
+      Set<CancellableRegionServerCallable> set =
           ((AsyncRequestFutureImpl<?>)reqs).getCallsInProgress();
       // verify we did cancel unneeded calls
       Assert.assertTrue(!set.isEmpty());
-      for (PayloadCarryingServerCallable m : set) {
+      for (CancellableRegionServerCallable m : set) {
         Assert.assertTrue(m.isCancelled());
       }
     } finally {