You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by li...@apache.org on 2014/05/20 20:18:49 UTC

svn commit: r1596351 [2/2] - in /hbase/branches/0.89-fb/src: main/java/org/apache/hadoop/hbase/client/ main/java/org/apache/hadoop/hbase/coprocessor/endpoints/ main/java/org/apache/hadoop/hbase/ipc/ main/java/org/apache/hadoop/hbase/ipc/thrift/ main/ja...

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/ipc/thrift/HBaseToThriftAdapter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/ipc/thrift/HBaseToThriftAdapter.java?rev=1596351&r1=1596350&r2=1596351&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/ipc/thrift/HBaseToThriftAdapter.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/ipc/thrift/HBaseToThriftAdapter.java Tue May 20 18:18:49 2014
@@ -26,6 +26,7 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.concurrent.ExecutionException;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -82,7 +83,7 @@ import com.google.common.util.concurrent
  */
 public class HBaseToThriftAdapter implements HRegionInterface {
   public static final Log LOG = LogFactory.getLog(HBaseToThriftAdapter.class);
-  public ThriftHRegionInterface connection;
+  public ThriftHRegionInterface.Async connection;
   public ThriftClientManager clientManager;
   private InetSocketAddress addr;
   private Configuration conf;
@@ -97,7 +98,7 @@ public class HBaseToThriftAdapter implem
       InetSocketAddress addr, Configuration conf,
       Class<? extends ThriftClientInterface> clazz,
       HBaseRPCOptions options) {
-    this.connection = (ThriftHRegionInterface)connection;
+    this.connection = (ThriftHRegionInterface.Async)connection;
     this.clientManager = clientManager;
     this.addr = addr;
     this.conf = conf;
@@ -120,18 +121,32 @@ public class HBaseToThriftAdapter implem
    */
   public void refreshConnectionAndThrowIOException(Exception e)
       throws IOException {
-    if (e instanceof TApplicationException) {
-      throw new RuntimeException(e);
-    } else if (e instanceof RuntimeTApplicationException) {
-      throw new RuntimeException(e);
+
+    Throwable throwable = e;
+    if (e instanceof ExecutionException) {
+      throwable = e.getCause();
+    }
+
+    if (throwable instanceof ThriftHBaseException) {
+      Exception serverCause  = ((ThriftHBaseException) throwable).getServerJavaException();
+      handleNotServingRegionException(serverCause);
+      handleIllegalArgumentException(serverCause);
+      handleIOException(serverCause);
+      throw new RuntimeException(serverCause);
+    }
+
+    if (throwable instanceof TApplicationException) {
+      throw new RuntimeException(throwable);
+    } else if (throwable instanceof RuntimeTApplicationException) {
+      throw new RuntimeException(throwable);
     } else {
       //TODO: creating a new connection is unnecessary.
       // We should replace it with cleanUpConnection later
       Pair<ThriftClientInterface, ThriftClientManager> interfaceAndManager = HBaseThriftRPC
           .refreshConnection(this.addr, this.conf, this.connection, this.clazz);
-      this.connection = (ThriftHRegionInterface) interfaceAndManager.getFirst();
+      this.connection = (ThriftHRegionInterface.Async) interfaceAndManager.getFirst();
       this.clientManager = interfaceAndManager.getSecond();
-      throw new IOException(e);
+      throw new IOException(throwable);
     }
   }
 
@@ -199,7 +214,7 @@ public class HBaseToThriftAdapter implem
       try {
         Pair<ThriftClientInterface, ThriftClientManager> clientAndManager = HBaseThriftRPC
             .getClientWithoutWrapper(addr, conf, clazz);
-        this.connection = (ThriftHRegionInterface) clientAndManager.getFirst();
+        this.connection = (ThriftHRegionInterface.Async) clientAndManager.getFirst();
         this.clientManager = clientAndManager.getSecond();
       } catch (Exception e) {
         throw new RuntimeException(e);
@@ -308,7 +323,7 @@ public class HBaseToThriftAdapter implem
   public boolean isStopped() {
     preProcess();
     try {
-      return connection.isStopped();
+      return connection.isStopped().get();
     } catch (Exception e) {
       refreshConnectionAndThrowRuntimeException(e);
       return false;
@@ -331,12 +346,7 @@ public class HBaseToThriftAdapter implem
       throws NotServingRegionException {
     preProcess();
     try {
-      return connection.getRegionInfo(regionName);
-    } catch (ThriftHBaseException te) {
-      Exception e = te.getServerJavaException();
-      handleNotServingRegionException(e);
-      LOG.warn("Unexpected Exception: " + e);
-      throw new RuntimeException(e);
+      return connection.getRegionInfo(regionName).get();
     } catch (Exception e) {
       refreshConnectionAndThrowRuntimeException(e);
       return null;
@@ -350,17 +360,12 @@ public class HBaseToThriftAdapter implem
       throws IOException {
     preProcess();
     try {
-      Result r = connection.getClosestRowBefore(regionName, row, family);
+      Result r = connection.getClosestRowBefore(regionName, row, family).get();
       if (r.isSentinelResult()) {
         return null;
       } else {
         return r;
       }
-    } catch (ThriftHBaseException te) {
-      Exception e = te.getServerJavaException();
-      handleIOException(e);
-      LOG.warn("Unexpected Exception: " + e);
-      throw new RuntimeException(e);
     } catch (Exception e) {
       refreshConnectionAndThrowIOException(e);
       return null;
@@ -371,7 +376,7 @@ public class HBaseToThriftAdapter implem
 
   public ListenableFuture<Result> getClosestRowBeforeAsync(byte[] regionName, byte[] row, byte[] family) {
     preProcess();
-    return connection.getClosestRowBeforeAsync(regionName, row, family);
+    return connection.getClosestRowBefore(regionName, row, family);
   }
 
   // TODO: we will decide whether to remove it from HRegionInterface in the future
@@ -386,12 +391,7 @@ public class HBaseToThriftAdapter implem
       IOException {
     preProcess();
     try {
-      connection.flushRegion(regionName);
-    } catch (ThriftHBaseException te) {
-      Exception e = te.getServerJavaException();
-      handleIllegalArgumentException(e);
-      handleIOException(e);
-      LOG.warn("Unexpected Exception: " + e);
+      connection.flushRegion(regionName).get();
     } catch (Exception e) {
       refreshConnectionAndThrowIOException(e);
     } finally {
@@ -404,12 +404,7 @@ public class HBaseToThriftAdapter implem
       throws IllegalArgumentException, IOException {
     preProcess();
     try {
-      connection.flushRegion(regionName, ifOlderThanTS);
-    } catch (ThriftHBaseException te) {
-      Exception e = te.getServerJavaException();
-      handleIllegalArgumentException(e);
-      handleIOException(e);
-      LOG.warn("Unexpected Exception: " + e);
+      connection.flushRegion(regionName, ifOlderThanTS).get();
     } catch (Exception e) {
       refreshConnectionAndThrowIOException(e);
     } finally {
@@ -421,7 +416,7 @@ public class HBaseToThriftAdapter implem
   public long getLastFlushTime(byte[] regionName) {
     preProcess();
     try {
-      return connection.getLastFlushTime(regionName);
+      return connection.getLastFlushTime(regionName).get();
     } catch (Exception e) {
       refreshConnectionAndThrowRuntimeException(e);
       return -1;
@@ -434,7 +429,7 @@ public class HBaseToThriftAdapter implem
   public MapWritable getLastFlushTimes() {
     preProcess();
     try {
-      Map<byte[], Long> map = connection.getLastFlushTimes();
+      Map<byte[], Long> map = connection.getLastFlushTimes().get();
       MapWritable writableMap = new MapWritable();
       for (Entry<byte[], Long> e : map.entrySet()) {
         writableMap.put(new BytesWritable(e.getKey()),
@@ -453,7 +448,7 @@ public class HBaseToThriftAdapter implem
   public long getCurrentTimeMillis() {
     preProcess();
     try {
-      return connection.getCurrentTimeMillis();
+      return connection.getCurrentTimeMillis().get();
     } catch (Exception e) {
       refreshConnectionAndThrowRuntimeException(e);
       return -1;
@@ -466,7 +461,7 @@ public class HBaseToThriftAdapter implem
   public long getStartCode() {
     preProcess();
     try {
-      return connection.getStartCode();
+      return connection.getStartCode().get();
     } catch (Exception e) {
       refreshConnectionAndThrowRuntimeException(e);
       return -1;
@@ -480,12 +475,7 @@ public class HBaseToThriftAdapter implem
       throws IllegalArgumentException {
     preProcess();
     try {
-      return connection.getStoreFileList(regionName, columnFamily);
-    } catch (ThriftHBaseException te) {
-      Exception e = te.getServerJavaException();
-      handleIllegalArgumentException(e);
-      LOG.warn("Unexpected Exception: " + e);
-      throw new RuntimeException(e);
+      return connection.getStoreFileList(regionName, columnFamily).get();
     } catch (Exception e) {
       refreshConnectionAndThrowRuntimeException(e);
       return null;
@@ -502,12 +492,7 @@ public class HBaseToThriftAdapter implem
       List<byte[]> columnFamiliesList = new ArrayList<>();
       Collections.addAll(columnFamiliesList, columnFamilies);
       return connection.getStoreFileListForColumnFamilies(regionName,
-          columnFamiliesList);
-    } catch (ThriftHBaseException te) {
-      Exception e = te.getServerJavaException();
-      handleIllegalArgumentException(e);
-      LOG.warn("Unexpected Exception: " + e);
-      throw new RuntimeException(e);
+          columnFamiliesList).get();
     } catch (Exception e) {
       refreshConnectionAndThrowRuntimeException(e);
       return null;
@@ -521,12 +506,7 @@ public class HBaseToThriftAdapter implem
       throws IllegalArgumentException {
     preProcess();
     try {
-      return connection.getStoreFileListForAllColumnFamilies(regionName);
-    } catch (ThriftHBaseException te) {
-      Exception e = te.getServerJavaException();
-      handleIllegalArgumentException(e);
-      LOG.warn("Unexpected Exception: " + e);
-      throw new RuntimeException(e);
+      return connection.getStoreFileListForAllColumnFamilies(regionName).get();
     } catch (Exception e) {
       refreshConnectionAndThrowRuntimeException(e);
       return null;
@@ -539,12 +519,7 @@ public class HBaseToThriftAdapter implem
   public List<String> getHLogsList(boolean rollCurrentHLog) throws IOException {
     preProcess();
     try {
-      return connection.getHLogsList(rollCurrentHLog);
-    } catch (ThriftHBaseException te) {
-      Exception e = te.getServerJavaException();
-      handleIOException(e);
-      LOG.warn("Unexpected Exception: " + e);
-      throw new RuntimeException(e);
+      return connection.getHLogsList(rollCurrentHLog).get();
     } catch (Exception e) {
       refreshConnectionAndThrowIOException(e);
       return null;
@@ -557,12 +532,7 @@ public class HBaseToThriftAdapter implem
   public Result get(byte[] regionName, Get get) throws IOException {
     preProcess();
     try {
-      return connection.get(regionName, get);
-    } catch (ThriftHBaseException te) {
-      Exception e = te.getServerJavaException();
-      handleIOException(e);
-      LOG.warn("Unexpected Exception: " + e);
-      throw new RuntimeException(e);
+      return connection.get(regionName, get).get();
     } catch (Exception e) {
       refreshConnectionAndThrowIOException(e);
       return null;
@@ -575,13 +545,8 @@ public class HBaseToThriftAdapter implem
   public Result[] get(byte[] regionName, List<Get> gets) throws IOException {
     preProcess();
     try {
-      List<Result> listOfResults = connection.getRows(regionName, gets);
+      List<Result> listOfResults = connection.getRows(regionName, gets).get();
       return listOfResults.toArray(new Result[listOfResults.size()]);
-    } catch (ThriftHBaseException te) {
-      Exception e = te.getServerJavaException();
-      handleIOException(e);
-      LOG.warn("Unexpected Exception: " + e);
-      throw new RuntimeException(e);
     } catch (Exception e) {
       refreshConnectionAndThrowIOException(e);
       return null;
@@ -592,19 +557,14 @@ public class HBaseToThriftAdapter implem
 
   public ListenableFuture<Result> getAsync(byte[] regionName, Get get) {
     preProcess();
-    return connection.getAsync(regionName, get);
+    return connection.get(regionName, get);
   }
 
   @Override
   public boolean exists(byte[] regionName, Get get) throws IOException {
     preProcess();
     try {
-      return connection.exists(regionName, get);
-    } catch (ThriftHBaseException te) {
-      Exception e = te.getServerJavaException();
-      handleIOException(e);
-      LOG.warn("Unexpected Exception: " + e);
-      return false;
+      return connection.exists(regionName, get).get();
     } catch (Exception e) {
       refreshConnectionAndThrowIOException(e);
       return false;
@@ -617,11 +577,7 @@ public class HBaseToThriftAdapter implem
   public void put(byte[] regionName, Put put) throws IOException {
     preProcess();
     try {
-      connection.put(regionName, put);
-    } catch (ThriftHBaseException te) {
-      Exception e = te.getServerJavaException();
-      handleIOException(e);
-      LOG.warn("Unexpected Exception: " + e);
+      connection.put(regionName, put).get();
     } catch (Exception e) {
       refreshConnectionAndThrowIOException(e);
     } finally {
@@ -633,12 +589,7 @@ public class HBaseToThriftAdapter implem
   public int put(byte[] regionName, List<Put> puts) throws IOException {
     preProcess();
     try {
-      return connection.putRows(regionName, puts);
-    } catch (ThriftHBaseException te) {
-      Exception e = te.getServerJavaException();
-      handleIOException(e);
-      LOG.warn("Unexpected Exception: " + e);
-      return -1;
+      return connection.putRows(regionName, puts).get();
     } catch (Exception e) {
       refreshConnectionAndThrowIOException(e);
       return -1;
@@ -651,11 +602,7 @@ public class HBaseToThriftAdapter implem
   public void delete(byte[] regionName, Delete delete) throws IOException {
     preProcess();
     try {
-      connection.processDelete(regionName, delete);
-    } catch (ThriftHBaseException te) {
-      Exception e = te.getServerJavaException();
-      handleIOException(e);
-      LOG.warn("Unexpected Exception: " + e);
+      connection.processDelete(regionName, delete).get();
     } catch (Exception e) {
       refreshConnectionAndThrowIOException(e);
     } finally {
@@ -665,7 +612,7 @@ public class HBaseToThriftAdapter implem
 
   public ListenableFuture<Void> deleteAsync(final byte[] regionName, final Delete delete) {
     preProcess();
-    return connection.deleteAsync(regionName, delete);
+    return connection.processDelete(regionName, delete);
   }
 
   @Override
@@ -673,13 +620,8 @@ public class HBaseToThriftAdapter implem
       throws IOException {
     preProcess();
     try {
-      return connection.processListOfDeletes(regionName, deletes);
-    } catch (ThriftHBaseException te) {
-      Exception e = te.getServerJavaException();
-      handleIOException(e);
-      LOG.warn("Unexpected Exception: " + e);
-      return -1;
-    } catch (Exception e) {
+      return connection.processListOfDeletes(regionName, deletes).get();
+    }  catch (Exception e) {
       refreshConnectionAndThrowIOException(e);
       return -1;
     } finally {
@@ -692,12 +634,7 @@ public class HBaseToThriftAdapter implem
       byte[] qualifier, byte[] value, Put put) throws IOException {
     preProcess();
     try {
-      return connection.checkAndPut(regionName, row, family, qualifier, value, put);
-    } catch (ThriftHBaseException te) {
-      Exception e = te.getServerJavaException();
-      handleIOException(e);
-      LOG.warn("Unexpected Exception: " + e);
-      return false;
+      return connection.checkAndPut(regionName, row, family, qualifier, value, put).get();
     } catch (Exception e) {
       refreshConnectionAndThrowIOException(e);
       return false;
@@ -711,13 +648,8 @@ public class HBaseToThriftAdapter implem
       byte[] qualifier, byte[] value, Delete delete) throws IOException {
     preProcess();
     try {
-      return connection.checkAndDelete(regionName, row, family, qualifier, value, delete);
-    } catch (ThriftHBaseException te) {
-      Exception e = te.getServerJavaException();
-      handleIOException(e);
-      LOG.warn("Unexpected Exception: " + e);
-      return false;
-    } catch (Exception e) {
+      return connection.checkAndDelete(regionName, row, family, qualifier, value, delete).get();
+    }  catch (Exception e) {
       refreshConnectionAndThrowIOException(e);
       return false;
     } finally {
@@ -731,12 +663,7 @@ public class HBaseToThriftAdapter implem
       throws IOException {
     preProcess();
     try {
-      return connection.incrementColumnValue(regionName, row, family, qualifier, amount, writeToWAL);
-    } catch (ThriftHBaseException te) {
-      Exception e = te.getServerJavaException();
-      handleIOException(e);
-      LOG.warn("Unexpected Exception: " + e);
-      return -1;
+      return connection.incrementColumnValue(regionName, row, family, qualifier, amount, writeToWAL).get();
     } catch (Exception e) {
       refreshConnectionAndThrowIOException(e);
       return -1;
@@ -749,12 +676,7 @@ public class HBaseToThriftAdapter implem
   public long openScanner(byte[] regionName, Scan scan) throws IOException {
     preProcess();
     try {
-      return connection.openScanner(regionName, scan);
-    } catch (ThriftHBaseException te) {
-      Exception e = te.getServerJavaException();
-      handleIOException(e);
-      LOG.warn("Unexpected Exception: " + e);
-      return -1;
+      return connection.openScanner(regionName, scan).get();
     } catch (Exception e) {
       refreshConnectionAndThrowIOException(e);
       return -1;
@@ -768,12 +690,8 @@ public class HBaseToThriftAdapter implem
       throws IOException {
     preProcess();
     try {
-      connection.mutateRow(regionName, TRowMutations.Builder.createFromRowMutations(arm));
-    } catch (ThriftHBaseException te) {
-      Exception e = te.getServerJavaException();
-      handleIOException(e);
-      LOG.warn("Unexpected Exception: " + e);
-    } catch (Exception e) {
+      connection.mutateRow(regionName, TRowMutations.Builder.createFromRowMutations(arm)).get();
+    }  catch (Exception e) {
       refreshConnectionAndThrowIOException(e);
     } finally {
       postProcess();
@@ -783,8 +701,9 @@ public class HBaseToThriftAdapter implem
   public ListenableFuture<Void> mutateRowAsync(byte[] regionName, RowMutations arm) {
     preProcess();
     try {
-      return connection.mutateRowAsync(regionName, TRowMutations.Builder.createFromRowMutations(arm));
+      return connection.mutateRow(regionName, TRowMutations.Builder.createFromRowMutations(arm));
     } catch (IOException e) {
+      // From creating thrift.
       return Futures.immediateFailedFuture(e);
     }
   }
@@ -798,11 +717,7 @@ public class HBaseToThriftAdapter implem
       for (RowMutations mutation : armList) {
         listOfMutations.add(TRowMutations.Builder.createFromRowMutations(mutation));
       }
-      connection.mutateRows(regionName, listOfMutations);
-    } catch (ThriftHBaseException te) {
-      Exception e = te.getServerJavaException();
-      handleIOException(e);
-      LOG.warn("Unexpected Exception: " + e);
+      connection.mutateRows(regionName, listOfMutations).get();
     } catch (Exception e) {
       refreshConnectionAndThrowIOException(e);
     } finally {
@@ -815,12 +730,7 @@ public class HBaseToThriftAdapter implem
   public Result next(long scannerId) throws IOException {
     preProcess();
     try {
-      return connection.next(scannerId);
-    } catch (ThriftHBaseException te) {
-      Exception e = te.getServerJavaException();
-      handleIOException(e);
-      LOG.warn("Unexpected Exception: " + e);
-      throw new RuntimeException(e);
+      return connection.next(scannerId).get();
     } catch (Exception e) {
       refreshConnectionAndThrowIOException(e);
       return null;
@@ -853,14 +763,9 @@ public class HBaseToThriftAdapter implem
   public Result[] next(long scannerId, int numberOfRows) throws IOException {
     preProcess();
     try {
-      List<Result> resultList = connection.nextRows(scannerId, numberOfRows);
+      List<Result> resultList = connection.nextRows(scannerId, numberOfRows).get();
       Result[] ret = resultList.toArray(new Result[resultList.size()]);
       return validateResults(ret);
-    } catch (ThriftHBaseException te) {
-      Exception e = te.getServerJavaException();
-      handleIOException(e);
-      LOG.warn("Unexpected Exception: " + e);
-      throw new RuntimeException(e);
     } catch (Exception e) {
       refreshConnectionAndThrowIOException(e);
       return null;
@@ -873,11 +778,7 @@ public class HBaseToThriftAdapter implem
   public void close(long scannerId) throws IOException {
     preProcess();
     try {
-      connection.close(scannerId);
-    } catch (ThriftHBaseException te) {
-      Exception e = te.getServerJavaException();
-      handleIOException(e);
-      LOG.warn("Unexpected Exception: " + e);
+      connection.close(scannerId).get();
     } catch (Exception e) {
       refreshConnectionAndThrowIOException(e);
     } finally {
@@ -889,12 +790,7 @@ public class HBaseToThriftAdapter implem
   public long lockRow(byte[] regionName, byte[] row) throws IOException {
     preProcess();
     try {
-      return connection.lockRow(regionName, row);
-    } catch (ThriftHBaseException te) {
-      Exception e = te.getServerJavaException();
-      handleIOException(e);
-      LOG.warn("Unexpected Exception: " + e);
-      return -1;
+      return connection.lockRow(regionName, row).get().getLockId();
     } catch (Exception e) {
       refreshConnectionAndThrowIOException(e);
       return -1;
@@ -905,19 +801,14 @@ public class HBaseToThriftAdapter implem
 
   public ListenableFuture<RowLock> lockRowAsync(byte[] regionName, byte[] row) {
     preProcess();
-    return connection.lockRowAsync(regionName, row);
+    return connection.lockRow(regionName, row);
   }
 
   @Override
   public void unlockRow(byte[] regionName, long lockId) throws IOException {
     preProcess();
     try {
-      connection.unlockRow(regionName, lockId);
-    } catch (ThriftHBaseException te) {
-      Exception e = te.getServerJavaException();
-      handleIOException(e);
-      LOG.warn("Unexpected Exception: " + e);
-      return;
+      connection.unlockRow(regionName, lockId).get();
     } catch (Exception e) {
       refreshConnectionAndThrowIOException(e);
     } finally {
@@ -927,20 +818,15 @@ public class HBaseToThriftAdapter implem
 
   public ListenableFuture<Void> unlockRowAsync(byte[] regionName, long lockId) {
     preProcess();
-    return connection.unlockRowAsync(regionName, lockId);
+    return connection.unlockRow(regionName, lockId);
   }
 
   @Override
   public HRegionInfo[] getRegionsAssignment() throws IOException {
     preProcess();
     try {
-      List<HRegionInfo> hRegionInfos = connection.getRegionsAssignment();
+      List<HRegionInfo> hRegionInfos = connection.getRegionsAssignment().get();
       return hRegionInfos.toArray(new HRegionInfo[hRegionInfos.size()]);
-    } catch (ThriftHBaseException te) {
-      Exception e = te.getServerJavaException();
-      handleIOException(e);
-      LOG.warn("Unexpected Exception: " + e);
-      throw new RuntimeException(e);
     } catch (Exception e) {
       refreshConnectionAndThrowIOException(e);
       return null;
@@ -953,12 +839,7 @@ public class HBaseToThriftAdapter implem
   public HServerInfo getHServerInfo() throws IOException {
     preProcess();
     try {
-      return connection.getHServerInfo();
-    } catch (ThriftHBaseException te) {
-      Exception e = te.getServerJavaException();
-      handleIOException(e);
-      LOG.warn("Unexpected Exception: " + e);
-      throw new RuntimeException(e);
+      return connection.getHServerInfo().get();
     } catch (Exception e) {
       refreshConnectionAndThrowIOException(e);
       return null;
@@ -972,12 +853,7 @@ public class HBaseToThriftAdapter implem
     preProcess();
     try {
       return MultiResponse.Builder.createFromTMultiResponse(connection
-          .multiAction(multi));
-    } catch (ThriftHBaseException te) {
-      Exception e = te.getServerJavaException();
-      handleIOException(e);
-      LOG.warn("Unexpected Exception: " + e);
-      throw new RuntimeException(e);
+          .multiAction(multi).get());
     } catch (Exception e) {
       refreshConnectionAndThrowIOException(e);
       return null;
@@ -990,12 +866,7 @@ public class HBaseToThriftAdapter implem
   public MultiPutResponse multiPut(MultiPut puts) throws IOException {
     preProcess();
     try {
-      return connection.multiPut(puts);
-    } catch (ThriftHBaseException te) {
-      Exception e = te.getServerJavaException();
-      handleIOException(e);
-      LOG.warn("Unexpected Exception: " + e);
-      throw new RuntimeException(e);
+      return connection.multiPut(puts).get();
     } catch (Exception e) {
       refreshConnectionAndThrowIOException(e);
       return null;
@@ -1009,11 +880,7 @@ public class HBaseToThriftAdapter implem
       byte[] familyName) throws IOException {
     preProcess();
     try {
-      connection.bulkLoadHFile(hfilePath, regionName, familyName);
-    } catch (ThriftHBaseException te) {
-      Exception e = te.getServerJavaException();
-      handleIOException(e);
-      LOG.warn("Unexpected Exception: " + e);
+      connection.bulkLoadHFile(hfilePath, regionName, familyName).get();
     } catch (Exception e) {
       refreshConnectionAndThrowIOException(e);
     } finally {
@@ -1026,11 +893,7 @@ public class HBaseToThriftAdapter implem
       byte[] familyName, boolean assignSeqNum) throws IOException {
     preProcess();
     try {
-      connection.bulkLoadHFile(hfilePath, regionName, familyName, assignSeqNum);
-    } catch (ThriftHBaseException te) {
-      Exception e = te.getServerJavaException();
-      handleIOException(e);
-      LOG.warn("Unexpected Exception: " + e);
+      connection.bulkLoadHFile(hfilePath, regionName, familyName, assignSeqNum).get();
     } catch (Exception e) {
       refreshConnectionAndThrowIOException(e);
     } finally {
@@ -1043,11 +906,7 @@ public class HBaseToThriftAdapter implem
       throws IOException {
     preProcess();
     try {
-      connection.closeRegion(hri, reportWhenCompleted);
-    } catch (ThriftHBaseException te) {
-      Exception e = te.getServerJavaException();
-      handleIOException(e);
-      LOG.warn("Unexpected Exception: " + e);
+      connection.closeRegion(hri, reportWhenCompleted).get();
     } catch (Exception e) {
       refreshConnectionAndThrowIOException(e);
     } finally {
@@ -1059,12 +918,7 @@ public class HBaseToThriftAdapter implem
   public int updateFavoredNodes(AssignmentPlan plan) throws IOException {
     preProcess();
     try {
-      return connection.updateFavoredNodes(plan);
-    } catch (ThriftHBaseException te) {
-      Exception e = te.getServerJavaException();
-      handleIOException(e);
-      LOG.warn("Unexpected Exception: " + e);
-      return -1;
+      return connection.updateFavoredNodes(plan).get();
     } catch (Exception e) {
       refreshConnectionAndThrowIOException(e);
       return -1;
@@ -1077,7 +931,7 @@ public class HBaseToThriftAdapter implem
   public void updateConfiguration() {
     preProcess();
     try {
-      connection.updateConfiguration();
+      connection.updateConfiguration().get();
     } catch (Exception e) {
       refreshConnectionAndThrowRuntimeException(e);
     } finally {
@@ -1089,7 +943,7 @@ public class HBaseToThriftAdapter implem
   public void stop(String why) {
     preProcess();
     try {
-      connection.stop(why);
+      connection.stop(why).get();
     } catch (Exception e) {
       refreshConnectionAndThrowRuntimeException(e);
     } finally {
@@ -1101,7 +955,7 @@ public class HBaseToThriftAdapter implem
   public String getStopReason() {
     preProcess();
     try {
-      return connection.getStopReason();
+      return connection.getStopReason().get();
     } catch (Exception e) {
       refreshConnectionAndThrowRuntimeException(e);
       throw new RuntimeException(e);
@@ -1114,7 +968,7 @@ public class HBaseToThriftAdapter implem
   public void setNumHDFSQuorumReadThreads(int maxThreads) {
     preProcess();
     try {
-      connection.setNumHDFSQuorumReadThreads(maxThreads);
+      connection.setNumHDFSQuorumReadThreads(maxThreads).get();
     } catch (Exception e) {
       refreshConnectionAndThrowRuntimeException(e);
     } finally {
@@ -1126,7 +980,7 @@ public class HBaseToThriftAdapter implem
   public void setHDFSQuorumReadTimeoutMillis(long timeoutMillis) {
     preProcess();
     try {
-      connection.setHDFSQuorumReadTimeoutMillis(timeoutMillis);
+      connection.setHDFSQuorumReadTimeoutMillis(timeoutMillis).get();
     } catch (Exception e) {
       refreshConnectionAndThrowRuntimeException(e);
     } finally {
@@ -1138,12 +992,7 @@ public class HBaseToThriftAdapter implem
   public String getConfProperty(String paramName) throws IOException {
     preProcess();
     try {
-      return connection.getConfProperty(paramName);
-    } catch (ThriftHBaseException te) {
-      Exception e = te.getServerJavaException();
-      handleIOException(e);
-      LOG.warn("Unexpected Exception: " + e);
-      return null;
+      return connection.getConfProperty(paramName).get();
     } catch (Exception e) {
       refreshConnectionAndThrowIOException(e);
       return null;
@@ -1156,16 +1005,11 @@ public class HBaseToThriftAdapter implem
   public List<Bucket> getHistogram(byte[] regionName) throws IOException {
     preProcess();
     try {
-       List<Bucket> buckets = connection.getHistogram(regionName);
+       List<Bucket> buckets = connection.getHistogram(regionName).get();
       if (buckets.isEmpty()) {
         return null;
       }
       return buckets;
-    } catch (ThriftHBaseException te) {
-      Exception e = te.getServerJavaException();
-      handleIOException(e);
-      LOG.warn("Unexpected Exception: " + e);
-      return null;
     } catch (Exception e) {
       refreshConnectionAndThrowIOException(e);
       return null;
@@ -1180,16 +1024,11 @@ public class HBaseToThriftAdapter implem
     preProcess();
     try {
       List<Bucket> buckets =
-          connection.getHistogramForStore(regionName, family);
+          connection.getHistogramForStore(regionName, family).get();
       if (buckets.isEmpty()) {
         return null;
       }
       return buckets;
-    } catch (ThriftHBaseException te) {
-      Exception e = te.getServerJavaException();
-      handleIOException(e);
-      LOG.warn("Unexpected Exception: " + e);
-      return null;
     } catch (Exception e) {
       refreshConnectionAndThrowIOException(e);
       return null;
@@ -1201,11 +1040,14 @@ public class HBaseToThriftAdapter implem
   @Override
   public byte[] callEndpoint(String epName, String methodName,
       List<byte[]> params, byte[] regionName, byte[] startRow,
-      byte[] stopRow) throws ThriftHBaseException {
+      byte[] stopRow) throws IOException {
     preProcess();
     try {
       return connection.callEndpoint(epName, methodName, params, regionName,
-          startRow, stopRow);
+          startRow, stopRow).get();
+    } catch (Exception e) {
+      refreshConnectionAndThrowIOException(e);
+      return null;
     } finally {
       postProcess();
     }
@@ -1216,12 +1058,7 @@ public class HBaseToThriftAdapter implem
     throws IOException {
     preProcess();
     try {
-      return connection.getHistograms(regionNames);
-    } catch (ThriftHBaseException te) {
-      Exception e = te.getServerJavaException();
-      handleIOException(e);
-      LOG.warn("Unexpected Exception: " + e);
-      return null;
+      return connection.getHistograms(regionNames).get();
     } catch (Exception e) {
       refreshConnectionAndThrowIOException(e);
       return null;
@@ -1235,12 +1072,7 @@ public class HBaseToThriftAdapter implem
     throws IOException {
     preProcess();
     try {
-      return connection.getLocation(table, row, reload);
-    } catch (ThriftHBaseException te) {
-      Exception e = te.getServerJavaException();
-      handleIOException(e);
-      LOG.warn("Unexpected Exception: " + e);
-      throw new RuntimeException(e);
+      return connection.getLocation(table, row, reload).get();
     } catch (Exception e) {
       refreshConnectionAndThrowIOException(e);
       return null;
@@ -1251,32 +1083,41 @@ public class HBaseToThriftAdapter implem
 
   @Override
   public ScannerResult scanOpen(byte[] regionName, Scan scan, int numberOfRows)
-      throws ThriftHBaseException {
+      throws IOException {
     preProcess();
     try {
-      return connection.scanOpen(regionName, scan, numberOfRows);
-    } finally {
+      return connection.scanOpen(regionName, scan, numberOfRows).get();
+    } catch (Exception e) {
+      refreshConnectionAndThrowIOException(e);
+      return null;
+    }  finally {
       postProcess();
     }
   }
 
   @Override
   public ScannerResult scanNext(long id, int numberOfRows)
-      throws ThriftHBaseException {
+      throws IOException {
     preProcess();
     try {
-      return connection.scanNext(id, numberOfRows);
+      return connection.scanNext(id, numberOfRows).get();
+    } catch (Exception e) {
+      refreshConnectionAndThrowIOException(e);
+      return null;
     } finally {
       postProcess();
     }
   }
 
   @Override
-  public boolean scanClose(long id) throws ThriftHBaseException {
+  public boolean scanClose(long id) throws IOException {
     preProcess();
     try {
-      return connection.scanClose(id);
-    } finally {
+      return connection.scanClose(id).get();
+    } catch (Exception e) {
+      refreshConnectionAndThrowIOException(e);
+      return false;
+    }  finally {
       postProcess();
     }
   }

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java?rev=1596351&r1=1596350&r2=1596351&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java Tue May 20 18:18:49 2014
@@ -4185,7 +4185,7 @@ public class HRegionServer implements HR
   @Override
   public byte[] callEndpoint(String epName, String methodName,
       List<byte[]> params, final byte[] regionName, final byte[] startRow,
-      final byte[] stopRow) throws ThriftHBaseException {
+      final byte[] stopRow) throws ThriftHBaseException, IOException {
     return endpointServer.callEndpoint(epName, methodName, params, regionName,
         startRow, stopRow);
   }

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/ThriftHRegionServer.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/ThriftHRegionServer.java?rev=1596351&r1=1596350&r2=1596351&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/ThriftHRegionServer.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/ThriftHRegionServer.java Tue May 20 18:18:49 2014
@@ -67,7 +67,7 @@ import com.google.common.util.concurrent
  * This is just a wrapper around {@link HRegionServer}
  *
  */
-public class ThriftHRegionServer implements ThriftHRegionInterface {
+public class ThriftHRegionServer implements ThriftHRegionInterface.Sync {
   private static Log LOG = LogFactory.getLog(ThriftHRegionServer.class);
 
   private HRegionServer server;
@@ -104,17 +104,6 @@ public class ThriftHRegionServer impleme
     }
   }
 
-  @Override
-  public ListenableFuture<Result> getClosestRowBeforeAsync(final byte[] regionName,
-      final byte[] row, final byte[] family) {
-    try {
-      Result result = getClosestRowBefore(regionName, row, family);
-      return Futures.immediateFuture(result);
-    } catch (ThriftHBaseException e) {
-      return Futures.immediateFailedFuture(e);
-    }
-  }
-
   /**
    * If there is a 'regioninfo' value in the result, it would be in Writable
    * serialized form. Add a thrift serialized HRegionInfo object for the
@@ -255,14 +244,6 @@ public class ThriftHRegionServer impleme
     }
   }
 
-  @Override
-  public ListenableFuture<Result> getAsync(byte[] regionName, Get get) {
-    try {
-      return Futures.immediateFuture(get(regionName, get));
-    } catch (ThriftHBaseException e) {
-      return Futures.immediateFailedFuture(e);
-    }
-  }
 
   @Override
   public List<Result> getRows(byte[] regionName, List<Get> gets)
@@ -317,16 +298,6 @@ public class ThriftHRegionServer impleme
   }
 
   @Override
-  public ListenableFuture<Void> deleteAsync(final byte[] regionName, final Delete delete) {
-    try {
-      processDelete(regionName, delete);
-      return Futures.immediateFuture(null);
-    } catch (ThriftHBaseException e) {
-      return Futures.immediateFailedFuture(e);
-    }
-  }
-
-  @Override
   public int processListOfDeletes(byte[] regionName, List<Delete> deletes)
       throws ThriftHBaseException {
     try {
@@ -404,16 +375,6 @@ public class ThriftHRegionServer impleme
   }
 
   @Override
-  public ListenableFuture<Void> mutateRowAsync(byte[] regionName, TRowMutations arm) {
-    try {
-      mutateRow(regionName, arm);
-      return Futures.immediateFuture(null);
-    } catch (ThriftHBaseException e) {
-      return Futures.immediateFailedFuture(e);
-    }
-  }
-
-  @Override
   @Deprecated
   public Result next(long scannerId) throws ThriftHBaseException {
     try {
@@ -448,26 +409,16 @@ public class ThriftHRegionServer impleme
   }
 
   @Override
-  public long lockRow(byte[] regionName, byte[] row)
+  public RowLock lockRow(byte[] regionName, byte[] row)
       throws ThriftHBaseException {
     try {
-      return server.lockRow(regionName, row);
+      return new RowLock(server.lockRow(regionName, row));
     } catch (IOException e) {
       throw new ThriftHBaseException(e);
     }
   }
 
   @Override
-  public ListenableFuture<RowLock> lockRowAsync(byte[] regionName, byte[] row) {
-    try {
-      long lockId = lockRow(regionName, row);
-      return Futures.immediateFuture(new RowLock(row, lockId));
-    } catch (ThriftHBaseException e) {
-      return Futures.immediateFailedFuture(e);
-    }
-  }
-
-  @Override
   public void unlockRow(byte[] regionName, long lockId)
       throws ThriftHBaseException {
     try {
@@ -478,16 +429,6 @@ public class ThriftHRegionServer impleme
   }
 
   @Override
-  public ListenableFuture<Void> unlockRowAsync(byte[] regionName, long lockId) {
-    try {
-      unlockRow(regionName, lockId);
-      return Futures.immediateFuture(null);
-    } catch (ThriftHBaseException e) {
-      return Futures.immediateFailedFuture(e);
-    }
-  }
-
-  @Override
   public List<HRegionInfo> getRegionsAssignment() throws ThriftHBaseException {
     try {
       return Arrays.asList(server.getRegionsAssignment());
@@ -636,7 +577,7 @@ public class ThriftHRegionServer impleme
   @Override
   public byte[] callEndpoint(String epName, String methodName,
       List<byte[]> params, final byte[] regionName, final byte[] startRow,
-      final byte[] stopRow) throws ThriftHBaseException {
+      final byte[] stopRow) throws ThriftHBaseException, IOException {
     return server.callEndpoint(epName, methodName, params, regionName,
         startRow, stopRow);
   }

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java?rev=1596351&r1=1596350&r2=1596351&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java Tue May 20 18:18:49 2014
@@ -615,7 +615,7 @@ public class HBaseTestingUtility {
     } else {
       new HBaseAdmin(getConfiguration()).createTable(desc, startKey, endKey,
           numRegions);
-    };
+    }
     return new HTableAsync(getConfiguration(), tableName);
   }
 

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java?rev=1596351&r1=1596350&r2=1596351&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java Tue May 20 18:18:49 2014
@@ -439,7 +439,7 @@ public class MiniHBaseCluster {
     return hbaseCluster.getRegionServer(serverNumber);
   }
 
-  public ThriftHRegionInterface getThriftRegionServer(int serverNumber) {
+  public ThriftHRegionInterface.Sync getThriftRegionServer(int serverNumber) {
     return new ThriftHRegionServer(getRegionServer(serverNumber));
   }
 

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/TestHServerInfo.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/TestHServerInfo.java?rev=1596351&r1=1596350&r2=1596351&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/TestHServerInfo.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/TestHServerInfo.java Tue May 20 18:18:49 2014
@@ -235,7 +235,7 @@ public class TestHServerInfo {
 
     HRegionInterface client = (HRegionInterface) HBaseThriftRPC.getClient(
         regionServer.getServerInfo().getServerAddress().getInetSocketAddress(),
-        conf, ThriftHRegionInterface.class, HBaseRPCOptions.DEFAULT);
+        conf, ThriftHRegionInterface.Async.class, HBaseRPCOptions.DEFAULT);
 
     assertEquals(regionServer.getHServerInfo(), client.getHServerInfo());
   }

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestFastFail.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestFastFail.java?rev=1596351&r1=1596350&r2=1596351&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestFastFail.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestFastFail.java Tue May 20 18:18:49 2014
@@ -32,6 +32,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HServerAddress;
+import org.apache.hadoop.hbase.LargeTests;
 import org.apache.hadoop.hbase.MediumTests;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.client.TableServers.FailureInfo;
@@ -55,13 +56,13 @@ import org.junit.experimental.categories
  * Sets up the HBase mini cluster once at start and runs through all client tests.
  * Each creates a table named for the method and does its stuff against that.
  */
-@Category(MediumTests.class)
+@Category(LargeTests.class)
 public class TestFastFail {
   private static final Log LOG = LogFactory.getLog(TestFastFail.class);
   private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
   private static byte [] FAMILY = Bytes.toBytes("testFamily");
   private static byte [] QUALIFIER = Bytes.toBytes("testQualifier");
-  private static int SLAVES = 15;
+  private static int SLAVES = 8;
 
   private static  byte [] TABLE = Bytes.toBytes("testFastFail");
   private static  int NUM_REGIONS = 100;

Added: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/ipc/TestThriftHRegionInterface.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/ipc/TestThriftHRegionInterface.java?rev=1596351&view=auto
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/ipc/TestThriftHRegionInterface.java (added)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/ipc/TestThriftHRegionInterface.java Tue May 20 18:18:49 2014
@@ -0,0 +1,145 @@
+/**
+ * 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.ipc;
+
+import com.facebook.swift.codec.ThriftField;
+import com.facebook.swift.service.ThriftMethod;
+import com.google.common.util.concurrent.ListenableFuture;
+import org.apache.hadoop.hbase.Restartable;
+import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.hbase.StopStatus;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.coprocessor.endpoints.IEndpointService;
+import org.apache.hadoop.hbase.regionserver.HRegionServerIf;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.lang.annotation.Annotation;
+import java.lang.reflect.Method;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import static org.junit.Assert.*;
+
+/**
+ * Test to make sure that our thrift interface is well defined and that Async and Sync interfaces are
+ * kept in sync.
+ */
+@Category(SmallTests.class)
+public class TestThriftHRegionInterface {
+
+  @Test
+  public void testMethodAnnotations() {
+    Map<String, Method> nameToSyncMethods = new HashMap<>(55);
+    Map<String, Method> nameToAsyncMethods = new HashMap<>(55);
+
+    for (Method m : ThriftHRegionInterface.Sync.class.getDeclaredMethods()) {
+      String thriftName = getThriftMethodName(m);
+      assertNotNull(thriftName);
+      assertNull("Each thrift named method should be unique. [ " + thriftName + " ]",
+          nameToSyncMethods.get(thriftName));
+      nameToSyncMethods.put(thriftName, m);
+      assertAnnotatedWell(m);
+    }
+
+    for (Method asyncMethod : ThriftHRegionInterface.Async.class.getDeclaredMethods()) {
+      String thriftName = getThriftMethodName(asyncMethod);
+      assertNotNull(thriftName);
+      Method syncMethod = nameToSyncMethods.get(thriftName);
+      assertNotNull("expecting " + thriftName + " to be in nameToSyncMethods", syncMethod);
+      assertNull("Each thrift named method should be unique. [ " + thriftName + " ]",
+          nameToAsyncMethods.get(thriftName));
+      nameToAsyncMethods.put(thriftName, asyncMethod);
+      assertAnnotatedWell(asyncMethod);
+      assertEquals(asyncMethod.getAnnotation(ThriftMethod.class),
+          syncMethod.getAnnotation(ThriftMethod.class));
+
+      Class<?> klass = asyncMethod.getReturnType();
+      assertEquals(
+          "Every method in the asyncMethod should return a future [ " + asyncMethod.getName()
+              + " ] = " + klass, ListenableFuture.class.getName(),
+          klass.getName());
+    }
+
+    assertEquals(ThriftHRegionInterface.Sync.class.getDeclaredMethods().length,
+        nameToSyncMethods.size());
+    assertEquals(ThriftHRegionInterface.Async.class.getDeclaredMethods().length,
+        nameToAsyncMethods.size());
+  }
+
+  @Test
+  public void testImplementsHRegionInterface() throws Exception {
+    assertEquals(HRegionInterface.class.getDeclaredMethods().length
+            + Restartable.class.getDeclaredMethods().length
+            + StopStatus.class.getDeclaredMethods().length
+            + IEndpointService.class.getDeclaredMethods().length
+            + IRegionScanService.class.getDeclaredMethods().length
+            - 1 /* getOnlineRegionsAsArray is test only*/,
+        ThriftHRegionInterface.Async.class.getDeclaredMethods().length);
+
+    assertEquals(HRegionInterface.class.getDeclaredMethods().length
+            + Restartable.class.getDeclaredMethods().length
+            + StopStatus.class.getDeclaredMethods().length
+            + IEndpointService.class.getDeclaredMethods().length
+            + IRegionScanService.class.getDeclaredMethods().length
+            - 1 /* getOnlineRegionsAsArray is test only*/,
+        ThriftHRegionInterface.Sync.class.getDeclaredMethods().length);
+
+    assertEquals(ThriftHRegionInterface.Sync.class.getDeclaredMethods().length,
+        ThriftHRegionInterface.Async.class.getDeclaredMethods().length);
+  }
+
+  protected void assertAnnotatedWell(Method m) {
+    ThriftMethod tm = m.getAnnotation(ThriftMethod.class);
+    assertNotNull("every method should have a ThriftMethod annotation", tm);
+    assertNotNull("There should be an exception annotation", tm.exception());
+    assertTrue("There should be an exception annotation", tm.exception().length > 0);
+
+    for (Annotation[] parameterAnnotations:m.getParameterAnnotations()) {
+
+      ThriftField thriftField = null;
+
+      for (Annotation a:parameterAnnotations) {
+        if (a instanceof ThriftField) {
+          thriftField = (ThriftField) a;
+        }
+      }
+
+      assertNotNull(thriftField);
+      assertNotNull(thriftField.name());
+      assertTrue("Every parameter should have a thriftField name", thriftField.name().length() > 1);
+    }
+  }
+
+  protected String getThriftMethodName(Method m) {
+    String thriftName = null;
+    Annotation[] annotations = m.getAnnotations();
+    for (Annotation a : annotations) {
+      if (a instanceof ThriftMethod) {
+        thriftName = ((ThriftMethod) a).value();
+      }
+    }
+    return thriftName;
+  }
+}

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/FailureInjectingThriftHRegionServer.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/FailureInjectingThriftHRegionServer.java?rev=1596351&r1=1596350&r2=1596351&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/FailureInjectingThriftHRegionServer.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/FailureInjectingThriftHRegionServer.java Tue May 20 18:18:49 2014
@@ -68,23 +68,23 @@ public class FailureInjectingThriftHRegi
   }
 
   @Override
-  public ListenableFuture<Result> getAsync(byte[] regionName, Get get) {
+  public Result get(byte[] regionName, Get get) throws ThriftHBaseException {
     switch (failureType) {
       // Region server will throw RegionOverloadedException, wrapped in ThriftHBaseException
       case REGIONOVERLOADEDEXCEPTION:
         if (++repeatCount > repeats) {
           clearExceptionMode();
-          return super.getAsync(regionName, get);
+          return super.get(regionName, get);
         }
         LOG.debug("Exception repeat count: " + repeatCount);
-        return Futures.immediateFailedFuture(new ThriftHBaseException(
-            new RegionOverloadedException("GetAsync Test ROE", HConstants.DEFAULT_HBASE_CLIENT_PAUSE)));
+        throw new ThriftHBaseException(
+            new RegionOverloadedException("GetAsync Test ROE", HConstants.DEFAULT_HBASE_CLIENT_PAUSE));
 
       // Region server will disconnect the client, cause TTransportException on client side
       case STOP:
         if (++repeatCount > repeats) {
           clearExceptionMode();
-          return super.getAsync(regionName, get);
+          return super.get(regionName, get);
         }
         LOG.debug("Exception repeat count: " + repeatCount);
         stop("GetAsync Test Stop");
@@ -98,22 +98,22 @@ public class FailureInjectingThriftHRegi
       case MIXEDRETRIABLEEXCEPTIONS:
         if (++repeatCount > repeats) {
           clearExceptionMode();
-          return super.getAsync(regionName, get);
+          return super.get(regionName, get);
         }
         LOG.debug("Exception repeat count: " + repeatCount);
-        return Futures.immediateFailedFuture(getRetriableExceptions(repeatCount));
+        throw new ThriftHBaseException(getRetriableExceptions(repeatCount));
 
       case DONOTRETRYEXCEPTION:
         if (++repeatCount > repeats) {
           clearExceptionMode();
-          return super.getAsync(regionName, get);
+          return super.get(regionName, get);
         }
         LOG.debug("Exception repeat count: " + repeatCount);
-        return Futures.immediateFailedFuture(new ThriftHBaseException(
-            new DoNotRetryIOException("GetAsync Test DoNotRetryIOE")));
+        throw new ThriftHBaseException(
+            new DoNotRetryIOException("GetAsync Test DoNotRetryIOE"));
 
       default:
-        return super.getAsync(regionName, get);
+        return super.get(regionName, get);
     }
   }
 

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionInfo.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionInfo.java?rev=1596351&r1=1596350&r2=1596351&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionInfo.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionInfo.java Tue May 20 18:18:49 2014
@@ -195,7 +195,7 @@ public class TestHRegionInfo {
         .getServerInfo().getHostname(),
        region.getRegionServer().getThriftServerPort());
     HRegionInterface server = (HRegionInterface) HBaseThriftRPC.getClient(addr,
-        conf, ThriftHRegionInterface.class, HBaseRPCOptions.DEFAULT);
+        conf, ThriftHRegionInterface.Async.class, HBaseRPCOptions.DEFAULT);
     HRegionInfo regionInfo = server.getRegionInfo(region.getRegionName());
     assertEquals(region.getRegionInfo(), regionInfo);
     testUtil.shutdownMiniCluster();

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/metrics/TestThriftMetrics.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/metrics/TestThriftMetrics.java?rev=1596351&r1=1596350&r2=1596351&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/metrics/TestThriftMetrics.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/metrics/TestThriftMetrics.java Tue May 20 18:18:49 2014
@@ -36,12 +36,14 @@ import org.apache.hadoop.hbase.util.Byte
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import static org.junit.Assert.assertTrue;
+
 /**
  * Test to verify that the thrift metrics are calculated and propagated in the
  * HBaseRpcMetrics.
  */
 @Category(MediumTests.class)
-public class TestThriftMetrics extends TestCase {
+public class TestThriftMetrics {
   private final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
   private final int SLAVES = 1;
   static final byte[] TABLE = Bytes.toBytes("testTable");
@@ -95,6 +97,7 @@ public class TestThriftMetrics extends T
    * Test if we report operations which are too slow and large.
    * @throws IOException
    */
+  @Test
   public void testAbnormalOperationsAreReported() throws IOException,
     InterruptedException {
     // Set the warning time to 0, so that we get tooSlow alarms triggered.

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/thrift/swift/TestHRegionInterfaceSimpleFunctions.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/thrift/swift/TestHRegionInterfaceSimpleFunctions.java?rev=1596351&r1=1596350&r2=1596351&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/thrift/swift/TestHRegionInterfaceSimpleFunctions.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/thrift/swift/TestHRegionInterfaceSimpleFunctions.java Tue May 20 18:18:49 2014
@@ -73,7 +73,7 @@ public class TestHRegionInterfaceSimpleF
       throws IOException, InterruptedException, ExecutionException, TimeoutException, ThriftHBaseException {
     HTable table = TEST_UTIL.createTable(TABLENAME, FAMILY);
     HRegionServer server = TEST_UTIL.getHBaseCluster().getRegionServer(0);
-    ThriftHRegionInterface thriftServer = TEST_UTIL.getHBaseCluster().getThriftRegionServer(0);
+    ThriftHRegionInterface.Sync thriftServer = TEST_UTIL.getHBaseCluster().getThriftRegionServer(0);
     TEST_UTIL.loadTable(table, FAMILY);
 
     HServerInfo info = server.getHServerInfo();
@@ -90,7 +90,7 @@ public class TestHRegionInterfaceSimpleF
     InetSocketAddress addr =
         new InetSocketAddress(info.getHostname(), server.getThriftServerPort());
     HRegionInterface client = (HRegionInterface) HBaseThriftRPC
-        .getClient(addr, TEST_UTIL.getConfiguration(), ThriftHRegionInterface.class, HBaseRPCOptions.DEFAULT);
+        .getClient(addr, TEST_UTIL.getConfiguration(), ThriftHRegionInterface.Async.class, HBaseRPCOptions.DEFAULT);
 
     // tGetClosestRowBefore
     Result r = null;
@@ -161,7 +161,7 @@ public class TestHRegionInterfaceSimpleF
   @Test
   public void testAtomicMutation() throws Exception {
     HTable table = TEST_UTIL.createTable(TABLENAME, FAMILY);
-    ThriftHRegionInterface thriftServer = TEST_UTIL.getHBaseCluster().getThriftRegionServer(0);
+    ThriftHRegionInterface.Sync thriftServer = TEST_UTIL.getHBaseCluster().getThriftRegionServer(0);
 
     byte[] row = Bytes.toBytes("test-row");
     byte[] invalidValue = Bytes.toBytes("test-row2");
@@ -190,7 +190,7 @@ public class TestHRegionInterfaceSimpleF
   @Test
   public void testQuorumConfigurationChanges() throws Exception {
     HRegionServer server = TEST_UTIL.getHBaseCluster().getRegionServer(0);
-    ThriftHRegionInterface thriftServer = TEST_UTIL.getHBaseCluster().getThriftRegionServer(0);
+    ThriftHRegionInterface.Sync thriftServer = TEST_UTIL.getHBaseCluster().getThriftRegionServer(0);
 
     int threads = server.getQuorumReadThreadsMax() + 1;
     long timeout = server.getQuorumReadTimeoutMillis() + 1;
@@ -208,7 +208,7 @@ public class TestHRegionInterfaceSimpleF
   @Test
   public void testCloseRegion() throws Exception {
     HRegionServer server = TEST_UTIL.getHBaseCluster().getRegionServer(0);
-    ThriftHRegionInterface thriftServer = TEST_UTIL.getHBaseCluster().getThriftRegionServer(0);
+    ThriftHRegionInterface.Sync thriftServer = TEST_UTIL.getHBaseCluster().getThriftRegionServer(0);
 
     HRegion[] region = server.getOnlineRegionsAsArray();
     HRegionInfo regionInfo = region[0].getRegionInfo();
@@ -235,7 +235,7 @@ public class TestHRegionInterfaceSimpleF
    */
   @Test
   public void testStopRegionServer() throws Exception {
-    ThriftHRegionInterface thriftServer = TEST_UTIL.getHBaseCluster().getThriftRegionServer(0);
+    ThriftHRegionInterface.Sync thriftServer = TEST_UTIL.getHBaseCluster().getThriftRegionServer(0);
     thriftServer.updateConfiguration();
 
     String why = "test reason";

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/thrift/swift/TestServerSideException.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/thrift/swift/TestServerSideException.java?rev=1596351&r1=1596350&r2=1596351&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/thrift/swift/TestServerSideException.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/thrift/swift/TestServerSideException.java Tue May 20 18:18:49 2014
@@ -73,7 +73,7 @@ public class TestServerSideException {
         true);
 
     conf.setClass(HConstants.THRIFT_REGION_SERVER_IMPL,
-        FailureInjectingThriftHRegionServer.class, ThriftHRegionInterface.class);
+        FailureInjectingThriftHRegionServer.class, ThriftHRegionInterface.Sync.class);
 
     conf.setInt(HConstants.CLIENT_RETRY_NUM_STRING, 5);
     // Server will allow client to retry once when there is RegionOverloadedException
@@ -211,7 +211,7 @@ public class TestServerSideException {
   public void testTTransportException() throws Exception {
     HTableAsync table = TEST_UTIL.createTable(new StringBytes("testTable2"),
         new byte[][] { FAMILY }, 3, Bytes.toBytes("bbb"),
-        Bytes.toBytes("yyy"), 25);
+        Bytes.toBytes("yyy"), 6);
 
     Put put = new Put(ROW);
     put.add(FAMILY, null, VALUE);

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/thrift/swift/TestThriftExceptions.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/thrift/swift/TestThriftExceptions.java?rev=1596351&r1=1596350&r2=1596351&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/thrift/swift/TestThriftExceptions.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/thrift/swift/TestThriftExceptions.java Tue May 20 18:18:49 2014
@@ -37,11 +37,15 @@ import org.junit.experimental.categories
 import java.io.IOException;
 import java.net.InetSocketAddress;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
 /**
  * This is a test class to verify if the Thrift Exceptions work as expected.
  */
 @Category(MediumTests.class)
-public class TestThriftExceptions extends TestCase {
+public class TestThriftExceptions {
   /**
    * Test if the ThriftHBaseException can be serialized and deserialized
    * correctly.
@@ -67,8 +71,7 @@ public class TestThriftExceptions extend
    * HBaseThriftAdapter.
    */
   @Test
-  public void testExceptionTranslation()
-    throws InterruptedException, IOException {
+  public void testExceptionTranslation() throws InterruptedException, IOException {
     HBaseTestingUtility testUtil = new HBaseTestingUtility();
     testUtil.getConfiguration().setBoolean(
       HConstants.REGION_SERVER_WRITE_THRIFT_INFO_TO_META, true);
@@ -83,13 +86,15 @@ public class TestThriftExceptions extend
 
     InetSocketAddress addr = new InetSocketAddress(port);
     HRegionInterface client = (HRegionInterface) HBaseThriftRPC.getClient(addr,
-      conf, ThriftHRegionInterface.class, HBaseRPCOptions.DEFAULT);
+      conf, ThriftHRegionInterface.Async.class, HBaseRPCOptions.DEFAULT);
 
     boolean illegalArgumentException = false;
     try {
       client.flushRegion(Bytes.toBytes("foobar"));
     } catch (IllegalArgumentException e) {
       illegalArgumentException = true;
+    } catch (Exception e) {
+      assertFalse(true);
     }
     assertTrue("Expected IllegalArgumentException", illegalArgumentException);