You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by nd...@apache.org on 2015/06/27 00:41:02 UTC

hbase git commit: HBASE-13214 Remove deprecated and unused methods from HTable class

Repository: hbase
Updated Branches:
  refs/heads/master 00cadf186 -> 3efde8561


HBASE-13214 Remove deprecated and unused methods from HTable class


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

Branch: refs/heads/master
Commit: 3efde85614e80967557a57b1461b22b101d8a946
Parents: 00cadf1
Author: Ashish Singhi <as...@huawei.com>
Authored: Fri Jun 26 23:48:19 2015 +0530
Committer: Nick Dimiduk <nd...@apache.org>
Committed: Fri Jun 26 15:39:53 2015 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hbase/MetaTableAccessor.java  |   2 +-
 .../hadoop/hbase/client/BufferedMutator.java    |   2 +-
 .../hbase/client/BufferedMutatorImpl.java       |   2 +-
 .../org/apache/hadoop/hbase/client/HTable.java  | 390 +------------------
 .../hadoop/hbase/client/HTableInterface.java    |  47 +--
 .../org/apache/hadoop/hbase/client/Table.java   |  31 +-
 .../hadoop/hbase/client/TestAsyncProcess.java   |   2 +-
 .../hadoop/hbase/rest/client/RemoteHTable.java  |  11 -
 .../hadoop/hbase/client/HTableWrapper.java      |  25 --
 .../regionserver/ReplicationSink.java           |   2 +-
 .../resources/hbase-webapps/master/table.jsp    |  32 +-
 .../hadoop/hbase/client/TestFromClientSide.java |  93 +++--
 .../hbase/client/TestFromClientSideNoCodec.java |   5 +-
 .../org/apache/hadoop/hbase/client/TestHCM.java |   6 +-
 .../hadoop/hbase/client/TestMultiParallel.java  |  35 +-
 .../hbase/client/TestRpcControllerFactory.java  |   2 +-
 .../hadoop/hbase/client/TestSizeFailures.java   |   4 +-
 .../hbase/coprocessor/TestHTableWrapper.java    |  39 +-
 .../coprocessor/TestOpenTableInCoprocessor.java |   2 +-
 19 files changed, 154 insertions(+), 578 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/3efde856/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 ee42b93..2fbfd9f 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
@@ -1457,7 +1457,7 @@ public class MetaTableAccessor {
       if (METALOG.isDebugEnabled()) {
         METALOG.debug(mutationsToString(mutations));
       }
-      t.batch(mutations, new Object[mutations.size()]);
+      t.batch(mutations, null);
     } catch (InterruptedException e) {
       InterruptedIOException ie = new InterruptedIOException(e.getMessage());
       ie.initCause(e);

http://git-wip-us.apache.org/repos/asf/hbase/blob/3efde856/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutator.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutator.java
index 4424cec..3287335 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutator.java
@@ -52,7 +52,7 @@ import java.util.List;
  * extreme circumstances, such as JVM or machine failure, may cause some data loss.</p>
  *
  * <p>NOTE: This class replaces the functionality that used to be available via
- * {@link HTableInterface#setAutoFlush(boolean)} set to {@code false}.
+ *HTableInterface#setAutoFlush(boolean) set to {@code false}.
  * </p>
  *
  * <p>See also the {@code BufferedMutatorExample} in the hbase-examples module.</p>

http://git-wip-us.apache.org/repos/asf/hbase/blob/3efde856/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 ba86986..2ee00d9 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
@@ -247,7 +247,7 @@ public class BufferedMutatorImpl implements BufferedMutator {
   }
 
   /**
-   * This is used for legacy purposes in {@link HTable#getWriteBuffer()} only. This should not beÓ
+   * This is used for legacy purposes only. This should not beÓ
    * called from production uses.
    * @deprecated Going away when we drop public support for {@link HTableInterface}.
 Ó   */

http://git-wip-us.apache.org/repos/asf/hbase/blob/3efde856/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 686aaa8..ebeaabd 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
@@ -24,7 +24,6 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
-import java.util.NavigableMap;
 import java.util.TreeMap;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutionException;
@@ -38,14 +37,10 @@ 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.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValueUtil;
-import org.apache.hadoop.hbase.MetaTableAccessor;
-import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
@@ -241,123 +236,6 @@ public class HTable implements HTableInterface {
   }
 
   /**
-   * Tells whether or not a table is enabled or not. This method creates a
-   * new HBase configuration, so it might make your unit tests fail due to
-   * incorrect ZK client port.
-   * @param tableName Name of table to check.
-   * @return {@code true} if table is online.
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated use {@link HBaseAdmin#isTableEnabled(byte[])}
-   */
-  @Deprecated
-  public static boolean isTableEnabled(String tableName) throws IOException {
-    return isTableEnabled(TableName.valueOf(tableName));
-  }
-
-  /**
-   * Tells whether or not a table is enabled or not. This method creates a
-   * new HBase configuration, so it might make your unit tests fail due to
-   * incorrect ZK client port.
-   * @param tableName Name of table to check.
-   * @return {@code true} if table is online.
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated use {@link HBaseAdmin#isTableEnabled(byte[])}
-   */
-  @Deprecated
-  public static boolean isTableEnabled(byte[] tableName) throws IOException {
-    return isTableEnabled(TableName.valueOf(tableName));
-  }
-
-  /**
-   * Tells whether or not a table is enabled or not. This method creates a
-   * new HBase configuration, so it might make your unit tests fail due to
-   * incorrect ZK client port.
-   * @param tableName Name of table to check.
-   * @return {@code true} if table is online.
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated use {@link HBaseAdmin#isTableEnabled(byte[])}
-   */
-  @Deprecated
-  public static boolean isTableEnabled(TableName tableName) throws IOException {
-    return isTableEnabled(HBaseConfiguration.create(), tableName);
-  }
-
-  /**
-   * Tells whether or not a table is enabled or not.
-   * @param conf The Configuration object to use.
-   * @param tableName Name of table to check.
-   * @return {@code true} if table is online.
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated use {@link HBaseAdmin#isTableEnabled(byte[])}
-   */
-  @Deprecated
-  public static boolean isTableEnabled(Configuration conf, String tableName)
-  throws IOException {
-    return isTableEnabled(conf, TableName.valueOf(tableName));
-  }
-
-  /**
-   * Tells whether or not a table is enabled or not.
-   * @param conf The Configuration object to use.
-   * @param tableName Name of table to check.
-   * @return {@code true} if table is online.
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated use {@link HBaseAdmin#isTableEnabled(byte[])}
-   */
-  @Deprecated
-  public static boolean isTableEnabled(Configuration conf, byte[] tableName)
-  throws IOException {
-    return isTableEnabled(conf, TableName.valueOf(tableName));
-  }
-
-  /**
-   * Tells whether or not a table is enabled or not.
-   * @param conf The Configuration object to use.
-   * @param tableName Name of table to check.
-   * @return {@code true} if table is online.
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated use {@link HBaseAdmin#isTableEnabled(org.apache.hadoop.hbase.TableName tableName)}
-   */
-  @Deprecated
-  public static boolean isTableEnabled(Configuration conf,
-      final TableName tableName) throws IOException {
-    try(Connection conn = ConnectionFactory.createConnection(conf)) {
-      return conn.getAdmin().isTableEnabled(tableName);
-    }
-  }
-
-  /**
-   * Find region location hosting passed row using cached info
-   * @param row Row to find.
-   * @return The location of the given row.
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated Use {@link RegionLocator#getRegionLocation(byte[])}
-   */
-  @Deprecated
-  public HRegionLocation getRegionLocation(final String row)
-  throws IOException {
-    return getRegionLocation(Bytes.toBytes(row), false);
-  }
-
-  /**
-   * @deprecated Use {@link RegionLocator#getRegionLocation(byte[])} instead.
-   */
-  @Deprecated
-  public HRegionLocation getRegionLocation(final byte [] row)
-  throws IOException {
-    return locator.getRegionLocation(row);
-  }
-
-  /**
-   * @deprecated Use {@link RegionLocator#getRegionLocation(byte[], boolean)} instead.
-   */
-  @Deprecated
-  public HRegionLocation getRegionLocation(final byte [] row, boolean reload)
-  throws IOException {
-    return locator.getRegionLocation(row, reload);
-  }
-
-  /**
    * {@inheritDoc}
    */
   @Override
@@ -384,15 +262,6 @@ public class HTable implements HTableInterface {
   }
 
   /**
-   * Kept in 0.96 for backward compatibility
-   * @deprecated  since 0.96. This is an internal buffer that should not be read nor write.
-   */
-  @Deprecated
-  public List<Row> getWriteBuffer() {
-    return mutator == null ? null : mutator.getWriteBuffer();
-  }
-
-  /**
    * {@inheritDoc}
    */
   @Override
@@ -457,69 +326,6 @@ public class HTable implements HTableInterface {
   }
 
   /**
-   * Gets all the regions and their address for this table.
-   * <p>
-   * This is mainly useful for the MapReduce integration.
-   * @return A map of HRegionInfo with it's server address
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated This is no longer a public API.  Use {@link #getAllRegionLocations()} instead.
-   */
-  @SuppressWarnings("deprecation")
-  @Deprecated
-  public NavigableMap<HRegionInfo, ServerName> getRegionLocations() throws IOException {
-    // TODO: Odd that this returns a Map of HRI to SN whereas getRegionLocator, singular,
-    // returns an HRegionLocation.
-    return MetaTableAccessor.allTableRegions(this.connection, getName());
-  }
-
-  /**
-   * Gets all the regions and their address for this table.
-   * <p>
-   * This is mainly useful for the MapReduce integration.
-   * @return A map of HRegionInfo with it's server address
-   * @throws IOException if a remote or network exception occurs
-   *
-   * @deprecated Use {@link RegionLocator#getAllRegionLocations()} instead;
-   */
-  @Deprecated
-  public List<HRegionLocation> getAllRegionLocations() throws IOException {
-    return locator.getAllRegionLocations();
-  }
-
-  /**
-   * Get the corresponding regions for an arbitrary range of keys.
-   * <p>
-   * @param startKey Starting row in range, inclusive
-   * @param endKey Ending row in range, exclusive
-   * @return A list of HRegionLocations corresponding to the regions that
-   * contain the specified range
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated This is no longer a public API
-   */
-  @Deprecated
-  public List<HRegionLocation> getRegionsInRange(final byte [] startKey,
-    final byte [] endKey) throws IOException {
-    return getRegionsInRange(startKey, endKey, false);
-  }
-
-  /**
-   * Get the corresponding regions for an arbitrary range of keys.
-   * <p>
-   * @param startKey Starting row in range, inclusive
-   * @param endKey Ending row in range, exclusive
-   * @param reload true to reload information or false to use cached information
-   * @return A list of HRegionLocations corresponding to the regions that
-   * contain the specified range
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated This is no longer a public API
-   */
-  @Deprecated
-  public List<HRegionLocation> getRegionsInRange(final byte [] startKey,
-      final byte [] endKey, final boolean reload) throws IOException {
-    return getKeysAndRegionsInRange(startKey, endKey, false, reload).getSecond();
-  }
-
-  /**
    * Get the corresponding start keys and regions for an arbitrary range of
    * keys.
    * <p>
@@ -529,9 +335,7 @@ public class HTable implements HTableInterface {
    * @return A pair of list of start keys and list of HRegionLocations that
    *         contain the specified range
    * @throws IOException if a remote or network exception occurs
-   * @deprecated This is no longer a public API
    */
-  @Deprecated
   private Pair<List<byte[]>, List<HRegionLocation>> getKeysAndRegionsInRange(
       final byte[] startKey, final byte[] endKey, final boolean includeEndKey)
       throws IOException {
@@ -549,9 +353,7 @@ public class HTable implements HTableInterface {
    * @return A pair of list of start keys and list of HRegionLocations that
    *         contain the specified range
    * @throws IOException if a remote or network exception occurs
-   * @deprecated This is no longer a public API
    */
-  @Deprecated
   private Pair<List<byte[]>, List<HRegionLocation>> getKeysAndRegionsInRange(
       final byte[] startKey, final byte[] endKey, final boolean includeEndKey,
       final boolean reload) throws IOException {
@@ -565,7 +367,7 @@ public class HTable implements HTableInterface {
     List<HRegionLocation> regionsInRange = new ArrayList<HRegionLocation>();
     byte[] currentKey = startKey;
     do {
-      HRegionLocation regionLocation = getRegionLocation(currentKey, reload);
+      HRegionLocation regionLocation = getRegionLocator().getRegionLocation(currentKey, reload);
       keysInRange.add(currentKey);
       regionsInRange.add(regionLocation);
       currentKey = regionLocation.getRegionInfo().getEndKey();
@@ -577,35 +379,6 @@ public class HTable implements HTableInterface {
   }
 
   /**
-   * {@inheritDoc}
-   * @deprecated Use reversed scan instead.
-   */
-   @Override
-   @Deprecated
-   public Result getRowOrBefore(final byte[] row, final byte[] family)
-       throws IOException {
-     RegionServerCallable<Result> callable = new RegionServerCallable<Result>(this.connection,
-         tableName, row) {
-       @Override
-      public Result call(int callTimeout) throws IOException {
-         PayloadCarryingRpcController controller = rpcControllerFactory.newController();
-         controller.setPriority(tableName);
-         controller.setCallTimeout(callTimeout);
-         ClientProtos.GetRequest request = RequestConverter.buildGetRowOrBeforeRequest(
-             getLocation().getRegionInfo().getRegionName(), row, family);
-         try {
-           ClientProtos.GetResponse response = getStub().get(controller, request);
-           if (!response.hasResult()) return null;
-           return ProtobufUtil.toResult(response.getResult());
-         } catch (ServiceException se) {
-           throw ProtobufUtil.getRemoteException(se);
-         }
-       }
-     };
-     return rpcCallerFactory.<Result>newCaller().callWithRetries(callable, this.operationTimeout);
-   }
-
-  /**
    * The underlying {@link HTable} must not be closed.
    * {@link HTableInterface#getScanner(Scan)} has other usage details.
    */
@@ -740,7 +513,8 @@ public class HTable implements HTableInterface {
       return new Result[]{get(gets.get(0))};
     }
     try {
-      Object [] r1 = batch((List)gets);
+      Object[] r1 = new Object[gets.size()];
+      batch((List) gets, r1);
 
       // translate.
       Result [] results = new Result[r1.length];
@@ -771,20 +545,6 @@ public class HTable implements HTableInterface {
 
   /**
    * {@inheritDoc}
-   * @deprecated If any exception is thrown by one of the actions, there is no way to
-   * retrieve the partially executed results. Use {@link #batch(List, Object[])} instead.
-   */
-  @Deprecated
-  @Override
-  public Object[] batch(final List<? extends Row> actions)
-     throws InterruptedException, IOException {
-    Object[] results = new Object[actions.size()];
-    batch(actions, results);
-    return results;
-  }
-
-  /**
-   * {@inheritDoc}
    */
   @Override
   public <R> void batchCallback(
@@ -795,23 +555,6 @@ public class HTable implements HTableInterface {
 
   /**
    * {@inheritDoc}
-   * @deprecated If any exception is thrown by one of the actions, there is no way to
-   * retrieve the partially executed results. Use
-   * {@link #batchCallback(List, Object[], Batch.Callback)}
-   * instead.
-   */
-  @Deprecated
-  @Override
-  public <R> Object[] batchCallback(
-    final List<? extends Row> actions, final Batch.Callback<R> callback) throws IOException,
-      InterruptedException {
-    Object[] results = new Object[actions.size()];
-    batchCallback(actions, results, callback);
-    return results;
-  }
-
-  /**
-   * {@inheritDoc}
    */
   @Override
   public void delete(final Delete delete)
@@ -1221,9 +964,9 @@ public class HTable implements HTableInterface {
       exists.add(ge);
     }
 
-    Object[] r1;
+    Object[] r1= new Object[exists.size()];
     try {
-      r1 = batch(exists);
+      batch(exists, r1);
     } catch (InterruptedException e) {
       throw (InterruptedIOException)new InterruptedIOException().initCause(e);
     }
@@ -1241,21 +984,6 @@ public class HTable implements HTableInterface {
 
   /**
    * {@inheritDoc}
-   * @deprecated Use {@link #existsAll(java.util.List)}  instead.
-   */
-  @Override
-  @Deprecated
-  public Boolean[] exists(final List<Get> gets) throws IOException {
-    boolean[] results = existsAll(gets);
-    Boolean[] objectResults = new Boolean[results.length];
-    for (int i = 0; i < results.length; ++i) {
-      objectResults[i] = results[i];
-    }
-    return objectResults;
-  }
-
-  /**
-   * {@inheritDoc}
    * @throws IOException
    */
   @Override
@@ -1353,19 +1081,6 @@ public class HTable implements HTableInterface {
 
   /**
    * {@inheritDoc}
-   * @deprecated in 0.96. When called with setAutoFlush(false), this function also
-   *  set clearBufferOnFail to true, which is unexpected but kept for historical reasons.
-   *  Replace it with setAutoFlush(false, false) if this is exactly what you want, or by
-   *  {@link #setAutoFlushTo(boolean)} for all other cases.
-   */
-  @Deprecated
-  @Override
-  public void setAutoFlush(boolean autoFlush) {
-    this.autoFlush = autoFlush;
-  }
-
-  /**
-   * {@inheritDoc}
    */
   @Override
   public void setAutoFlushTo(boolean autoFlush) {
@@ -1419,101 +1134,6 @@ public class HTable implements HTableInterface {
   }
 
   /**
-   * Enable or disable region cache prefetch for the table. It will be
-   * applied for the given table's all HTable instances who share the same
-   * connection. By default, the cache prefetch is enabled.
-   * @param tableName name of table to configure.
-   * @param enable Set to true to enable region cache prefetch. Or set to
-   * false to disable it.
-   * @throws IOException
-   * @deprecated does nothing since 0.99
-   */
-  @Deprecated
-  public static void setRegionCachePrefetch(final byte[] tableName,
-      final boolean enable)  throws IOException {
-  }
-
-  /**
-   * @deprecated does nothing since 0.99
-   */
-  @Deprecated
-  public static void setRegionCachePrefetch(
-      final TableName tableName,
-      final boolean enable) throws IOException {
-  }
-
-  /**
-   * Enable or disable region cache prefetch for the table. It will be
-   * applied for the given table's all HTable instances who share the same
-   * connection. By default, the cache prefetch is enabled.
-   * @param conf The Configuration object to use.
-   * @param tableName name of table to configure.
-   * @param enable Set to true to enable region cache prefetch. Or set to
-   * false to disable it.
-   * @throws IOException
-   * @deprecated does nothing since 0.99
-   */
-  @Deprecated
-  public static void setRegionCachePrefetch(final Configuration conf,
-      final byte[] tableName, final boolean enable) throws IOException {
-  }
-
-  /**
-   * @deprecated does nothing since 0.99
-   */
-  @Deprecated
-  public static void setRegionCachePrefetch(final Configuration conf,
-      final TableName tableName,
-      final boolean enable) throws IOException {
-  }
-
-  /**
-   * Check whether region cache prefetch is enabled or not for the table.
-   * @param conf The Configuration object to use.
-   * @param tableName name of table to check
-   * @return true if table's region cache prefecth is enabled. Otherwise
-   * it is disabled.
-   * @throws IOException
-   * @deprecated always return false since 0.99
-   */
-  @Deprecated
-  public static boolean getRegionCachePrefetch(final Configuration conf,
-      final byte[] tableName) throws IOException {
-    return false;
-  }
-
-  /**
-   * @deprecated always return false since 0.99
-   */
-  @Deprecated
-  public static boolean getRegionCachePrefetch(final Configuration conf,
-      final TableName tableName) throws IOException {
-    return false;
-  }
-
-  /**
-   * Check whether region cache prefetch is enabled or not for the table.
-   * @param tableName name of table to check
-   * @return true if table's region cache prefecth is enabled. Otherwise
-   * it is disabled.
-   * @throws IOException
-   * @deprecated always return false since 0.99
-   */
-  @Deprecated
-  public static boolean getRegionCachePrefetch(final byte[] tableName) throws IOException {
-    return false;
-  }
-
-  /**
-   * @deprecated always return false since 0.99
-   */
-  @Deprecated
-  public static boolean getRegionCachePrefetch(
-      final TableName tableName) throws IOException {
-    return false;
-  }
-
-  /**
    * Explicitly clears the region cache to fetch the latest value from META.
    * This is a power user function: avoid unless you know the ramifications.
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/3efde856/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java
index 745c770..4cd81e7 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java
@@ -19,7 +19,6 @@
 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.hbase.classification.InterfaceStability;
@@ -46,26 +45,6 @@ public interface HTableInterface extends Table {
   byte[] getTableName();
 
   /**
-   * @deprecated Use {@link #existsAll(java.util.List)}  instead.
-   */
-  @Deprecated
-  Boolean[] exists(List<Get> gets) throws IOException;
-
-
-  /**
-   * See {@link #setAutoFlush(boolean, boolean)}
-   *
-   * @param autoFlush
-   *          Whether or not to enable 'auto-flush'.
-   * @deprecated in 0.96. When called with setAutoFlush(false), this function also
-   *  set clearBufferOnFail to true, which is unexpected but kept for historical reasons.
-   *  Replace it with setAutoFlush(false, false) if this is exactly what you want, though
-   *  this is the method you want for most cases.
-   */
-  @Deprecated
-  void setAutoFlush(boolean autoFlush);
-
-  /**
    * Turns 'auto-flush' on or off.
    * <p>
    * When enabled (default), {@link Put} operations don't get buffered/delayed
@@ -96,8 +75,7 @@ public interface HTableInterface extends Table {
    *          Whether to keep Put failures in the writeBuffer. If autoFlush is true, then
    *          the value of this parameter is ignored and clearBufferOnFail is set to true.
    *          Setting clearBufferOnFail to false is deprecated since 0.96.
-   * @deprecated in 0.99 since setting clearBufferOnFail is deprecated. Use
-   *  {@link #setAutoFlush(boolean)}} instead.
+   * @deprecated in 0.99 since setting clearBufferOnFail is deprecated.
    * @see BufferedMutator#flush()
    */
   @Deprecated
@@ -105,8 +83,8 @@ public interface HTableInterface extends Table {
 
   /**
    * Set the autoFlush behavior, without changing the value of {@code clearBufferOnFail}.
-   * @deprecated in 0.99 since setting clearBufferOnFail is deprecated. Use
-   * {@link #setAutoFlush(boolean)} instead, or better still, move on to {@link BufferedMutator}
+   * @deprecated in 0.99 since setting clearBufferOnFail is deprecated. Move on to
+   *             {@link BufferedMutator}
    */
   @Deprecated
   void setAutoFlushTo(boolean autoFlush);
@@ -157,23 +135,4 @@ public interface HTableInterface extends Table {
    */
   @Deprecated
   void setWriteBufferSize(long writeBufferSize) throws IOException;
-
-
-  /**
-   * Return the row that matches <i>row</i> exactly,
-   * or the one that immediately precedes it.
-   *
-   * @param row A row key.
-   * @param family Column family to include in the {@link Result}.
-   * @throws IOException if a remote or network exception occurs.
-   * @since 0.20.0
-   *
-   * @deprecated As of version 0.92 this method is deprecated without
-   * replacement. Since version 0.96+, you can use reversed scan.
-   * getRowOrBefore is used internally to find entries in hbase:meta and makes
-   * various assumptions about the table (which are true for hbase:meta but not
-   * in general) to be efficient.
-   */
-  @Deprecated
-  Result getRowOrBefore(byte[] row, byte[] family) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/3efde856/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 498c587..3e9db00 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
@@ -120,21 +120,6 @@ public interface Table extends Closeable {
     InterruptedException;
 
   /**
-   * Same as {@link #batch(List, Object[])}, but returns an array of
-   * results instead of using a results parameter reference.
-   *
-   * @param actions list of Get, Put, Delete, Increment, Append objects
-   * @return the results from the actions. A null in the return array means that
-   *         the call for that action failed, even after retries
-   * @throws IOException
-   * @since 0.90.0
-   * @deprecated If any exception is thrown by one of the actions, there is no way to
-   * retrieve the partially executed results. Use {@link #batch(List, Object[])} instead.
-   */
-  @Deprecated
-  Object[] batch(final List<? extends Row> actions) throws IOException, InterruptedException;
-
-  /**
    * Same as {@link #batch(List, Object[])}, but with a callback.
    * @since 0.96.0
    */
@@ -144,20 +129,6 @@ public interface Table extends Closeable {
     throws IOException, InterruptedException;
 
   /**
-   * Same as {@link #batch(List)}, but with a callback.
-   *
-   * @since 0.96.0
-   * @deprecated If any exception is thrown by one of the actions, there is no way to retrieve the
-   * partially executed results. Use {@link #batchCallback(List, Object[],
-   * org.apache.hadoop.hbase.client.coprocessor.Batch.Callback)} instead.
-   */
-  @Deprecated
-  <R> Object[] batchCallback(
-    List<? extends Row> actions, Batch.Callback<R> callback
-  ) throws IOException,
-    InterruptedException;
-
-  /**
    * Extracts certain cells from a given row.
    * @param get The object that specifies what data to fetch and from which row.
    * @return The data coming from the specified row, if it exists.  If the row
@@ -219,7 +190,7 @@ public interface Table extends Closeable {
 
   /**
    * Puts some data in the table.
-   * 
+   *
    * @param put The data to put.
    * @throws IOException if a remote or network exception occurs.
    * @since 0.20.0

http://git-wip-us.apache.org/repos/asf/hbase/blob/3efde856/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 1460de4..8340f97 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
@@ -867,7 +867,7 @@ public class TestAsyncProcess {
     MyAsyncProcess ap = new MyAsyncProcess(con, conf, con.nbThreads);
     ht.multiAp = ap;
 
-    ht.batch(gets, new Object[gets.size()]);
+    ht.batch(gets, null);
 
     Assert.assertEquals(ap.nbActions.get(), NB_REGS);
     Assert.assertEquals("1 multi response per server", 2, ap.nbMultiResponse.get());

http://git-wip-us.apache.org/repos/asf/hbase/blob/3efde856/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 2d122df..a12c747 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
@@ -792,23 +792,12 @@ public class RemoteHTable implements Table {
   }
 
   @Override
-  public Object[] batch(List<? extends Row> actions) throws IOException {
-    throw new IOException("batch not supported");
-  }
-
-  @Override
   public <R> void batchCallback(List<? extends Row> actions, Object[] results,
       Batch.Callback<R> callback) throws IOException, InterruptedException {
     throw new IOException("batchCallback not supported");
   }
 
   @Override
-  public <R> Object[] batchCallback(List<? extends Row> actions, Batch.Callback<R> callback)
-   throws IOException, InterruptedException {
-    throw new IOException("batchCallback not supported");
-  }
-
-  @Override
   public CoprocessorRpcChannel coprocessorService(byte[] row) {
     throw new UnsupportedOperationException("coprocessorService not implemented");
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/3efde856/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 e3641c7..6713546 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
@@ -242,37 +242,12 @@ public final class HTableWrapper implements Table {
     table.batch(actions, results);
   }
 
-  /**
-   * {@inheritDoc}
-   * @deprecated If any exception is thrown by one of the actions, there is no way to
-   * retrieve the partially executed results. Use {@link #batch(List, Object[])} instead.
-   */
-  @Deprecated
-  @Override
-  public Object[] batch(List<? extends Row> actions)
-      throws IOException, InterruptedException {
-    return table.batch(actions);
-  }
-
   @Override
   public <R> void batchCallback(List<? extends Row> actions, Object[] results,
       Batch.Callback<R> callback) throws IOException, InterruptedException {
     table.batchCallback(actions, results, callback);
   }
 
-  /**
-   * {@inheritDoc}
-   * @deprecated If any exception is thrown by one of the actions, there is no way to
-   * retrieve the partially executed results. Use
-   * {@link #batchCallback(List, Object[], Batch.Callback)} instead.
-   */
-  @Deprecated
-  @Override
-  public <R> Object[] batchCallback(List<? extends Row> actions,
-      Batch.Callback<R> callback) throws IOException, InterruptedException {
-    return table.batchCallback(actions, callback);
-  }
-
   @Override
   public Result[] get(List<Get> gets) throws IOException {
     return table.get(gets);

http://git-wip-us.apache.org/repos/asf/hbase/blob/3efde856/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
index 7d47677..fc7aa8e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
@@ -234,7 +234,7 @@ public class ReplicationSink {
     try {
       table = this.sharedHtableCon.getTable(tableName);
       for (List<Row> rows : allRows) {
-        table.batch(rows);
+        table.batch(rows, null);
       }
     } catch (InterruptedException ix) {
       throw (InterruptedIOException)new InterruptedIOException().initCause(ix);

http://git-wip-us.apache.org/repos/asf/hbase/blob/3efde856/hbase-server/src/main/resources/hbase-webapps/master/table.jsp
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/resources/hbase-webapps/master/table.jsp b/hbase-server/src/main/resources/hbase-webapps/master/table.jsp
index f057a44..45be52b 100644
--- a/hbase-server/src/main/resources/hbase-webapps/master/table.jsp
+++ b/hbase-server/src/main/resources/hbase-webapps/master/table.jsp
@@ -20,18 +20,21 @@
 <%@ page contentType="text/html;charset=UTF-8"
   import="static org.apache.commons.lang.StringEscapeUtils.escapeXml"
   import="java.util.TreeMap"
+  import="java.util.List"
   import="java.util.Map"
   import="java.util.Set"
   import="java.util.Collection"
   import="org.apache.hadoop.conf.Configuration"
   import="org.apache.hadoop.hbase.client.HTable"
   import="org.apache.hadoop.hbase.client.Admin"
+  import="org.apache.hadoop.hbase.client.RegionLocator"
   import="org.apache.hadoop.hbase.HRegionInfo"
+  import="org.apache.hadoop.hbase.HRegionLocation"
   import="org.apache.hadoop.hbase.ServerName"
   import="org.apache.hadoop.hbase.ServerLoad"
   import="org.apache.hadoop.hbase.RegionLoad"
   import="org.apache.hadoop.hbase.HConstants"
-  import="org.apache.hadoop.hbase.master.HMaster" 
+  import="org.apache.hadoop.hbase.master.HMaster"
   import="org.apache.hadoop.hbase.zookeeper.MetaTableLocator"
   import="org.apache.hadoop.hbase.util.Bytes"
   import="org.apache.hadoop.hbase.util.FSUtils"
@@ -119,8 +122,8 @@
         </div><!--/.nav-collapse -->
     </div>
 </div>
-<% 
-if ( fqtn != null ) { 
+<%
+if ( fqtn != null ) {
   table = (HTable) master.getConnection().getTable(fqtn);
   if (table.getTableDescriptor().getRegionReplication() > 1) {
     tableHeader = "<h2>Table Regions</h2><table class=\"table table-striped\"><tr><th>Name</th><th>Region Server</th><th>Start Key</th><th>End Key</th><th>Locality</th><th>Requests</th><th>ReplicaID</th></tr>";
@@ -128,7 +131,7 @@ if ( fqtn != null ) {
   } else {
     tableHeader = "<h2>Table Regions</h2><table class=\"table table-striped\"><tr><th>Name</th><th>Region Server</th><th>Start Key</th><th>End Key</th><th>Locality</th><th>Requests</th></tr>";
   }
-  if ( !readOnly && action != null ) { 
+  if ( !readOnly && action != null ) {
 %>
 <div class="container">
 
@@ -147,7 +150,7 @@ if ( fqtn != null ) {
       } else {
         admin.split(TableName.valueOf(fqtn));
       }
-    
+
     %> Split request accepted. <%
     } else if (action.equals("compact")) {
       if (key != null && key.length() > 0) {
@@ -201,6 +204,7 @@ if ( fqtn != null ) {
 </table>
 <%} else {
   Admin admin = master.getConnection().getAdmin();
+  RegionLocator r = master.getConnection().getRegionLocator(table.getName());
   try { %>
 <h2>Table Attributes</h2>
 <table class="table table-striped">
@@ -256,7 +260,7 @@ if ( fqtn != null ) {
     <table class="table table-striped">
       <tr>
        <th>Property</th>
-       <th>Value</th>       
+       <th>Value</th>
       </tr>
     <%
     Map<Bytes, Bytes> familyValues = family.getValues();
@@ -278,13 +282,13 @@ if ( fqtn != null ) {
 </table>
 <%
   Map<ServerName, Integer> regDistribution = new TreeMap<ServerName, Integer>();
-  Map<HRegionInfo, ServerName> regions = table.getRegionLocations();
+  List<HRegionLocation> regions = r.getAllRegionLocations();
   if(regions != null && regions.size() > 0) { %>
 <%=     tableHeader %>
 <%
-  for (Map.Entry<HRegionInfo, ServerName> hriEntry : regions.entrySet()) {
-    HRegionInfo regionInfo = hriEntry.getKey();
-    ServerName addr = hriEntry.getValue();
+  for (HRegionLocation hriEntry : regions) {
+    HRegionInfo regionInfo = hriEntry.getRegionInfo();
+    ServerName addr = hriEntry.getServerName();
     long req = 0;
     float locality = 0.0f;
     String urlRegionServer = null;
@@ -339,8 +343,8 @@ if ( fqtn != null ) {
 <h2>Regions by Region Server</h2>
 <table class="table table-striped"><tr><th>Region Server</th><th>Region Count</th></tr>
 <%
-  for (Map.Entry<ServerName, Integer> rdEntry : regDistribution.entrySet()) {   
-     ServerName addr = rdEntry.getKey();                                       
+  for (Map.Entry<ServerName, Integer> rdEntry : regDistribution.entrySet()) {
+     ServerName addr = rdEntry.getKey();
      String url = "//" + addr.getHostname() + ":" + master.getRegionServerInfoPort(addr) + "/";
 %>
 <tr>
@@ -399,8 +403,8 @@ Actions:
 <% } %>
 </div>
 </div>
-<% } 
-} else { // handle the case for fqtn is null with error message + redirect 
+<% }
+} else { // handle the case for fqtn is null with error message + redirect
 %>
 <div class="container">
     <div class="row inner_header">

http://git-wip-us.apache.org/repos/asf/hbase/blob/3efde856/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 d945cce..1b3fcf2 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
@@ -28,7 +28,6 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
-import java.lang.reflect.Method;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -43,12 +42,10 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.atomic.AtomicReference;
 
-import org.apache.log4j.Level;
 import org.apache.commons.lang.ArrayUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
@@ -93,7 +90,6 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType;
 import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MultiRowMutationService;
 import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MutateRowsRequest;
-import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.NoSuchColumnFamilyException;
 import org.apache.hadoop.hbase.regionserver.Region;
@@ -103,8 +99,8 @@ import org.apache.hadoop.hbase.testclassification.LargeTests;
 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.ZooKeeperWatcher;
 import org.apache.log4j.AppenderSkeleton;
+import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
 import org.apache.log4j.spi.LoggingEvent;
 import org.junit.After;
@@ -3824,7 +3820,7 @@ public class TestFromClientSide {
     // Only do this test if it is a HTable
     if(t instanceof HTableInterface) {
       HTable table = (HTable) t;
-      table.setAutoFlush(false);
+      table.setAutoFlushTo(false);
       ArrayList<Put> rowsUpdate = new ArrayList<Put>();
       for (int i = 0; i < NB_BATCH_ROWS * 10; i++) {
         byte[] row = Bytes.toBytes("row" + i);
@@ -4211,55 +4207,83 @@ public class TestFromClientSide {
         region.flush(true);
 
         Result result;
+        Get get = null;
 
         // Test before first that null is returned
-        result = table.getRowOrBefore(beforeFirstRow, HConstants.CATALOG_FAMILY);
-        assertTrue(result == null);
+        get = new Get(beforeFirstRow);
+        get.setClosestRowBefore(true);
+        get.addFamily(HConstants.CATALOG_FAMILY);
+        result = table.get(get);
+        assertTrue(result.isEmpty());
 
         // Test at first that first is returned
-        result = table.getRowOrBefore(firstRow, HConstants.CATALOG_FAMILY);
+        get = new Get(firstRow);
+        get.setClosestRowBefore(true);
+        get.addFamily(HConstants.CATALOG_FAMILY);
+        result = table.get(get);
         assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
         assertTrue(Bytes.equals(result.getRow(), firstRow));
         assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), one));
 
         // Test in between first and second that first is returned
-        result = table.getRowOrBefore(beforeSecondRow, HConstants.CATALOG_FAMILY);
+        get = new Get(beforeSecondRow);
+        get.setClosestRowBefore(true);
+        get.addFamily(HConstants.CATALOG_FAMILY);
+        result = table.get(get);
         assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
         assertTrue(Bytes.equals(result.getRow(), firstRow));
         assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), one));
 
         // Test at second make sure second is returned
-        result = table.getRowOrBefore(secondRow, HConstants.CATALOG_FAMILY);
+        get = new Get(secondRow);
+        get.setClosestRowBefore(true);
+        get.addFamily(HConstants.CATALOG_FAMILY);
+        result = table.get(get);
         assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
         assertTrue(Bytes.equals(result.getRow(), secondRow));
         assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), two));
 
         // Test in second and third, make sure second is returned
-        result = table.getRowOrBefore(beforeThirdRow, HConstants.CATALOG_FAMILY);
+        get = new Get(beforeThirdRow);
+        get.setClosestRowBefore(true);
+        get.addFamily(HConstants.CATALOG_FAMILY);
+        result = table.get(get);
         assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
         assertTrue(Bytes.equals(result.getRow(), secondRow));
         assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), two));
 
         // Test at third make sure third is returned
-        result = table.getRowOrBefore(thirdRow, HConstants.CATALOG_FAMILY);
+        get = new Get(thirdRow);
+        get.setClosestRowBefore(true);
+        get.addFamily(HConstants.CATALOG_FAMILY);
+        result = table.get(get);
         assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
         assertTrue(Bytes.equals(result.getRow(), thirdRow));
         assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), three));
 
         // Test in third and forth, make sure third is returned
-        result = table.getRowOrBefore(beforeForthRow, HConstants.CATALOG_FAMILY);
+        get = new Get(beforeForthRow);
+        get.setClosestRowBefore(true);
+        get.addFamily(HConstants.CATALOG_FAMILY);
+        result = table.get(get);
         assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
         assertTrue(Bytes.equals(result.getRow(), thirdRow));
         assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), three));
 
         // Test at forth make sure forth is returned
-        result = table.getRowOrBefore(forthRow, HConstants.CATALOG_FAMILY);
+        get = new Get(forthRow);
+        get.setClosestRowBefore(true);
+        get.addFamily(HConstants.CATALOG_FAMILY);
+        result = table.get(get);
         assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
         assertTrue(Bytes.equals(result.getRow(), forthRow));
         assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), four));
 
         // Test after forth make sure forth is returned
-        result = table.getRowOrBefore(Bytes.add(forthRow, one), HConstants.CATALOG_FAMILY);
+        get = new Get(Bytes.add(forthRow, one));
+        get.setClosestRowBefore(true);
+        get.addFamily(HConstants.CATALOG_FAMILY);
+        result = table.get(get);
         assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
         assertTrue(Bytes.equals(result.getRow(), forthRow));
         assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), four));
@@ -4571,7 +4595,7 @@ public class TestFromClientSide {
     assertIncrementKey(kvs[0], ROW, FAMILY, QUALIFIERS[0], 2);
     assertIncrementKey(kvs[1], ROW, FAMILY, QUALIFIERS[1], 2);
     assertIncrementKey(kvs[2], ROW, FAMILY, QUALIFIERS[2], 2);
-    
+
     ht.close();
   }
 
@@ -5269,49 +5293,64 @@ public class TestFromClientSide {
       assertEquals(26, numOfRegions);
 
       // Get the regions in this range
-      List<HRegionLocation> regionsList = table.getRegionsInRange(startKey,
-          endKey);
+      List<HRegionLocation> regionsList = getRegionsInRange(TABLE, startKey, endKey);
       assertEquals(10, regionsList.size());
 
       // Change the start key
       startKey = Bytes.toBytes("fff");
-      regionsList = table.getRegionsInRange(startKey, endKey);
+      regionsList = getRegionsInRange(TABLE, startKey, endKey);
       assertEquals(7, regionsList.size());
 
       // Change the end key
       endKey = Bytes.toBytes("nnn");
-      regionsList = table.getRegionsInRange(startKey, endKey);
+      regionsList = getRegionsInRange(TABLE, startKey, endKey);
       assertEquals(8, regionsList.size());
 
       // Empty start key
-      regionsList = table.getRegionsInRange(HConstants.EMPTY_START_ROW, endKey);
+      regionsList = getRegionsInRange(TABLE, HConstants.EMPTY_START_ROW, endKey);
       assertEquals(13, regionsList.size());
 
       // Empty end key
-      regionsList = table.getRegionsInRange(startKey, HConstants.EMPTY_END_ROW);
+      regionsList = getRegionsInRange(TABLE, startKey, HConstants.EMPTY_END_ROW);
       assertEquals(21, regionsList.size());
 
       // Both start and end keys empty
-      regionsList = table.getRegionsInRange(HConstants.EMPTY_START_ROW,
+      regionsList = getRegionsInRange(TABLE, HConstants.EMPTY_START_ROW,
           HConstants.EMPTY_END_ROW);
       assertEquals(26, regionsList.size());
 
       // Change the end key to somewhere in the last block
       endKey = Bytes.toBytes("zzz1");
-      regionsList = table.getRegionsInRange(startKey, endKey);
+      regionsList = getRegionsInRange(TABLE, startKey, endKey);
       assertEquals(21, regionsList.size());
 
       // Change the start key to somewhere in the first block
       startKey = Bytes.toBytes("aac");
-      regionsList = table.getRegionsInRange(startKey, endKey);
+      regionsList = getRegionsInRange(TABLE, startKey, endKey);
       assertEquals(26, regionsList.size());
 
       // Make start and end key the same
       startKey = endKey = Bytes.toBytes("ccc");
-      regionsList = table.getRegionsInRange(startKey, endKey);
+      regionsList = getRegionsInRange(TABLE, startKey, endKey);
       assertEquals(1, regionsList.size());
     }
   }
+  
+  private List<HRegionLocation> getRegionsInRange(TableName tableName, byte[] startKey,
+      byte[] endKey) throws IOException {
+    List<HRegionLocation> regionsInRange = new ArrayList<HRegionLocation>();
+    byte[] currentKey = startKey;
+    final boolean endKeyIsEndOfTable = Bytes.equals(endKey, HConstants.EMPTY_END_ROW);
+    try (RegionLocator r = TEST_UTIL.getConnection().getRegionLocator(tableName);) {
+      do {
+        HRegionLocation regionLocation = r.getRegionLocation(currentKey);
+        regionsInRange.add(regionLocation);
+        currentKey = regionLocation.getRegionInfo().getEndKey();
+      } while (!Bytes.equals(currentKey, HConstants.EMPTY_END_ROW)
+          && (endKeyIsEndOfTable || Bytes.compareTo(currentKey, endKey) < 0));
+      return regionsInRange;
+    }
+  }
 
   @Test
   public void testJira6912() throws Exception {

http://git-wip-us.apache.org/repos/asf/hbase/blob/3efde856/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideNoCodec.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideNoCodec.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideNoCodec.java
index 323d2f5..d92d301 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideNoCodec.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideNoCodec.java
@@ -84,7 +84,10 @@ public class TestFromClientSideNoCodec {
       HTableInterface hti = (HTableInterface) ht;
       // Check getRowOrBefore
       byte[] f = fs[0];
-      r = hti.getRowOrBefore(row, f);
+      Get get = new Get(row);
+      get.setClosestRowBefore(true);
+      get.addFamily(f);
+      r = ht.get(get);
       assertTrue(r.toString(), r.containsColumn(f, f));
     }
     // Check scan.

http://git-wip-us.apache.org/repos/asf/hbase/blob/3efde856/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 e946b21..fc53e1f 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
@@ -28,6 +28,7 @@ import java.io.IOException;
 import java.lang.reflect.Field;
 import java.lang.reflect.Modifier;
 import java.net.SocketTimeoutException;
+import java.util.ArrayList;
 import java.util.List;
 import java.util.Random;
 import java.util.concurrent.ExecutorService;
@@ -982,8 +983,9 @@ public class TestHCM {
        Put put4 = new Put(otherRow);
        put4.add(FAM_NAM, otherRow, otherRow);
 
-       // do multi
-       table.batch(Lists.newArrayList(put4, put3)); // first should be a valid row,
+      // do multi
+      ArrayList<Put> actions = Lists.newArrayList(put4, put3);
+      table.batch(actions, null); // first should be a valid row,
        // second we get RegionMovedException.
 
        setNumTries(conn, prevNumRetriesVal);

http://git-wip-us.apache.org/repos/asf/hbase/blob/3efde856/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 95faf1a..63b60bb 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
@@ -150,7 +150,7 @@ public class TestMultiParallel {
       try {
         try (Table t = connection.getTable(TEST_TABLE, executor)) {
           List<Put> puts = constructPutRequests(); // creates a Put for every region
-          t.batch(puts);
+          t.batch(puts, null);
           HashSet<ServerName> regionservers = new HashSet<ServerName>();
           try (RegionLocator locator = connection.getRegionLocator(TEST_TABLE)) {
             for (Row r : puts) {
@@ -173,7 +173,7 @@ public class TestMultiParallel {
 
     // load test data
     List<Put> puts = constructPutRequests();
-    table.batch(puts);
+    table.batch(puts, null);
 
     // create a list of gets and run it
     List<Row> gets = new ArrayList<Row>();
@@ -326,7 +326,8 @@ public class TestMultiParallel {
     // put multiple rows using a batch
     List<Put> puts = constructPutRequests();
 
-    Object[] results = table.batch(puts);
+    Object[] results = new Object[puts.size()];
+    table.batch(puts, results);
     validateSizeAndEmpty(results, KEYS.length);
 
     if (true) {
@@ -337,7 +338,8 @@ public class TestMultiParallel {
       liveRS.getRegionServer().abort("Aborting for tests", new Exception("testBatchWithPut"));
       puts = constructPutRequests();
       try {
-        results = table.batch(puts);
+        results = new Object[puts.size()];
+        table.batch(puts, results);
       } catch (RetriesExhaustedWithDetailsException ree) {
         LOG.info(ree.getExhaustiveDescription());
         table.close();
@@ -357,7 +359,8 @@ public class TestMultiParallel {
 
     // Load some data
     List<Put> puts = constructPutRequests();
-    Object[] results = table.batch(puts);
+    Object[] results = new Object[puts.size()];
+    table.batch(puts, results);
     validateSizeAndEmpty(results, KEYS.length);
 
     // Deletes
@@ -367,7 +370,8 @@ public class TestMultiParallel {
       delete.addFamily(BYTES_FAMILY);
       deletes.add(delete);
     }
-    results = table.batch(deletes);
+    results= new Object[deletes.size()];
+    table.batch(deletes, results);
     validateSizeAndEmpty(results, KEYS.length);
 
     // Get to make sure ...
@@ -386,7 +390,8 @@ public class TestMultiParallel {
 
     // Load some data
     List<Put> puts = constructPutRequests();
-    Object[] results = table.batch(puts);
+    Object[] results = new Object[puts.size()];
+    table.batch(puts, results);
     validateSizeAndEmpty(results, KEYS.length);
 
     // Deletes
@@ -420,7 +425,8 @@ public class TestMultiParallel {
       put.add(BYTES_FAMILY, qual, VALUE);
       puts.add(put);
     }
-    Object[] results = table.batch(puts);
+    Object[] results = new Object[puts.size()];
+    table.batch(puts, results);
 
     // validate
     validateSizeAndEmpty(results, 100);
@@ -434,7 +440,8 @@ public class TestMultiParallel {
       gets.add(get);
     }
 
-    Object[] multiRes = table.batch(gets);
+    Object[] multiRes = new Object[gets.size()];
+    table.batch(gets, multiRes);
 
     int idx = 0;
     for (Object r : multiRes) {
@@ -471,7 +478,8 @@ public class TestMultiParallel {
     actions.add(inc);
     actions.add(a);
 
-    Object[] multiRes = table.batch(actions);
+    Object[] multiRes = new Object[actions.size()];
+    table.batch(actions, multiRes);
     validateResult(multiRes[1], QUAL1, Bytes.toBytes("abcdef"));
     validateResult(multiRes[1], QUAL4, Bytes.toBytes("xyz"));
     validateResult(multiRes[0], QUAL2, Bytes.toBytes(2L));
@@ -577,7 +585,9 @@ public class TestMultiParallel {
     Table table = UTIL.getConnection().getTable(TEST_TABLE);
 
     // Load some data to start
-    Object[] results = table.batch(constructPutRequests());
+    List<Put> puts = constructPutRequests();
+    Object[] results = new Object[puts.size()];
+    table.batch(puts, results);
     validateSizeAndEmpty(results, KEYS.length);
 
     // Batch: get, get, put(new col), delete, get, get of put, get of deleted,
@@ -621,7 +631,8 @@ public class TestMultiParallel {
     put.add(BYTES_FAMILY, qual2, val2);
     actions.add(put);
 
-    results = table.batch(actions);
+    results = new Object[actions.size()];
+    table.batch(actions, results);
 
     // Validation
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/3efde856/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRpcControllerFactory.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRpcControllerFactory.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRpcControllerFactory.java
index dcf26f2..fca34c1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRpcControllerFactory.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRpcControllerFactory.java
@@ -148,7 +148,7 @@ public class TestRpcControllerFactory {
 
     Put p2 = new Put(row);
     p2.add(fam1, Bytes.toBytes("qual"), Bytes.toBytes("val1"));
-    table.batch(Lists.newArrayList(p, p2), new Object[2]);
+    table.batch(Lists.newArrayList(p, p2), null);
     // this only goes to a single server, so we don't need to change the count here
     counter = verifyCount(counter);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/3efde856/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSizeFailures.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSizeFailures.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSizeFailures.java
index 71b2d72..eedb57c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSizeFailures.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSizeFailures.java
@@ -92,13 +92,13 @@ public class TestSizeFailures {
         puts.add(p);
 
         if (puts.size() == 1000) {
-          table.batch(puts, new Object[1000]);
+          table.batch(puts, null);
           puts.clear();
         }
       }
 
       if (puts.size() > 0) {
-        table.batch(puts, new Object[puts.size()]);
+        table.batch(puts, null);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/3efde856/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestHTableWrapper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestHTableWrapper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestHTableWrapper.java
index 8efbfbf..c1d21fa 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestHTableWrapper.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestHTableWrapper.java
@@ -19,9 +19,16 @@
 
 package org.apache.hadoop.hbase.coprocessor;
 
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Coprocessor;
@@ -54,8 +61,6 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import static org.junit.Assert.*;
-
 /**
  * Tests class {@link org.apache.hadoop.hbase.client.HTableWrapper}
  * by invoking its methods and briefly asserting the result is reasonable.
@@ -257,9 +262,12 @@ public class TestHTableWrapper {
   }
 
   private void checkBatch() throws IOException, InterruptedException {
-    Object[] results1 = hTableInterface.batch(Arrays.asList(new Row[] {
-        new Increment(ROW_A).addColumn(TEST_FAMILY, qualifierCol1, 2L),
-        new Increment(ROW_A).addColumn(TEST_FAMILY, qualifierCol1, 2L) }));
+    List<Row> actions =
+        Arrays.asList(new Row[] { new Increment(ROW_A).addColumn(TEST_FAMILY, qualifierCol1, 2L),
+            new Increment(ROW_A).addColumn(TEST_FAMILY, qualifierCol1, 2L) });
+    Object[] results3 = new Object[actions.size()];
+    Object[] results1 = results3;
+    hTableInterface.batch(actions, results1);
     assertEquals(2, results1.length);
     for (Object r2 : results1) {
       assertTrue(r2 instanceof Result);
@@ -267,8 +275,7 @@ public class TestHTableWrapper {
     checkRowValue(ROW_A, Bytes.toBytes(0L));
     Object[] results2 = new Result[2];
     hTableInterface.batch(
-        Arrays.asList(new Row[] { new Increment(ROW_A).addColumn(TEST_FAMILY, qualifierCol1, 2L),
-            new Increment(ROW_A).addColumn(TEST_FAMILY, qualifierCol1, 2L) }), results2);
+        actions, results2);
     for (Object r2 : results2) {
       assertTrue(r2 instanceof Result);
     }
@@ -276,15 +283,12 @@ public class TestHTableWrapper {
 
     // with callbacks:
     final long[] updateCounter = new long[] { 0L };
-    Object[] results3 = hTableInterface.batchCallback(
-        Arrays.asList(new Row[] { new Increment(ROW_A).addColumn(TEST_FAMILY, qualifierCol1, 2L),
-            new Increment(ROW_A).addColumn(TEST_FAMILY, qualifierCol1, 2L) }),
-        new Batch.Callback<Result>() {
-          @Override
-          public void update(byte[] region, byte[] row, Result result) {
-            updateCounter[0]++;
-          }
-        });
+    hTableInterface.batchCallback(actions, results3, new Batch.Callback<Result>() {
+      @Override
+      public void update(byte[] region, byte[] row, Result result) {
+        updateCounter[0]++;
+      }
+    });
     assertEquals(2, updateCounter[0]);
     assertEquals(2, results3.length);
     for (Object r3 : results3) {
@@ -295,8 +299,7 @@ public class TestHTableWrapper {
     Object[] results4 = new Result[2];
     updateCounter[0] = 0L;
     hTableInterface.batchCallback(
-        Arrays.asList(new Row[] { new Increment(ROW_A).addColumn(TEST_FAMILY, qualifierCol1, 2L),
-            new Increment(ROW_A).addColumn(TEST_FAMILY, qualifierCol1, 2L) }), results4,
+        actions, results4,
         new Batch.Callback<Result>() {
           @Override
           public void update(byte[] region, byte[] row, Result result) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/3efde856/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestOpenTableInCoprocessor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestOpenTableInCoprocessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestOpenTableInCoprocessor.java
index 4c3594b..3122b4c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestOpenTableInCoprocessor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestOpenTableInCoprocessor.java
@@ -104,7 +104,7 @@ public class TestOpenTableInCoprocessor {
       Put p = new Put(new byte[] { 'a' });
       p.add(family, null, new byte[] { 'a' });
       try {
-        table.batch(Collections.singletonList(put));
+        table.batch(Collections.singletonList(put), null);
       } catch (InterruptedException e1) {
         throw new IOException(e1);
       }