You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ch...@apache.org on 2017/09/28 12:30:37 UTC

[07/19] hbase git commit: HBASE-18839 Apply RegionInfo to code base

http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/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 6c401a9..c7a1aff 100644
--- a/hbase-server/src/main/resources/hbase-webapps/master/table.jsp
+++ b/hbase-server/src/main/resources/hbase-webapps/master/table.jsp
@@ -17,54 +17,57 @@
  * limitations under the License.
  */
 --%>
-<%@page import="org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType"%>
+<%@page import="java.net.URLEncoder"%>
 <%@ page contentType="text/html;charset=UTF-8"
   import="static org.apache.commons.lang3.StringEscapeUtils.escapeXml"
-  import="org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString"
-  import="java.net.URLEncoder"
   import="java.util.ArrayList"
-  import="java.util.TreeMap"
-  import="java.util.List"
-  import="java.util.LinkedHashMap"
-  import="java.util.Map"
-  import="java.util.Set"
   import="java.util.Collection"
   import="java.util.Collections"
   import="java.util.Comparator"
+  import="java.util.LinkedHashMap"
+  import="java.util.List"
+  import="java.util.Map"
+  import="java.util.TreeMap"
   import="org.apache.commons.lang3.StringEscapeUtils"
   import="org.apache.hadoop.conf.Configuration"
-  import="org.apache.hadoop.util.StringUtils"
-  import="org.apache.hadoop.hbase.HRegionInfo"
+  import="org.apache.hadoop.hbase.HBaseConfiguration"
+  import="org.apache.hadoop.hbase.HColumnDescriptor"
+  import="org.apache.hadoop.hbase.HConstants"
   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.ServerLoad"
+  import="org.apache.hadoop.hbase.ServerName"
+  import="org.apache.hadoop.hbase.TableName"
+  import="org.apache.hadoop.hbase.TableNotFoundException"
+  import="org.apache.hadoop.hbase.client.Admin"
+  import="org.apache.hadoop.hbase.client.CompactionState"
+  import="org.apache.hadoop.hbase.client.RegionInfo"
+  import="org.apache.hadoop.hbase.client.RegionInfoBuilder"
+  import="org.apache.hadoop.hbase.client.RegionLocator"
+  import="org.apache.hadoop.hbase.client.RegionReplicaUtil"
+  import="org.apache.hadoop.hbase.client.Table"
   import="org.apache.hadoop.hbase.master.HMaster"
-  import="org.apache.hadoop.hbase.zookeeper.MetaTableLocator"
   import="org.apache.hadoop.hbase.quotas.QuotaTableUtil"
   import="org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot"
   import="org.apache.hadoop.hbase.util.Bytes"
   import="org.apache.hadoop.hbase.util.FSUtils"
-  import="org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos"
-  import="org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos"
-  import="org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas"
-  import="org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota"
-  import="org.apache.hadoop.hbase.TableName"
-  import="org.apache.hadoop.hbase.HColumnDescriptor"
-  import="org.apache.hadoop.hbase.HBaseConfiguration"
-  import="org.apache.hadoop.hbase.TableNotFoundException"%>
-<%@ page import="org.apache.hadoop.hbase.client.*" %>
+  import="org.apache.hadoop.hbase.zookeeper.MetaTableLocator"
+  import="org.apache.hadoop.util.StringUtils"
+  import="org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString"%>
+<%@ page import="org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos" %>
+<%@ page import="org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos" %>
+<%@ page import="org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas" %>
+<%@ page import="org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota" %>
 <%!
   /**
-   * @return An empty region load stamped with the passed in <code>hri</code>
+   * @return An empty region load stamped with the passed in <code>regionInfo</code>
    * region name.
    */
-  private RegionLoad getEmptyRegionLoad(final HRegionInfo hri) {
+  private RegionLoad getEmptyRegionLoad(final RegionInfo regionInfo) {
     return new RegionLoad(ClusterStatusProtos.RegionLoad.newBuilder().
       setRegionSpecifier(HBaseProtos.RegionSpecifier.newBuilder().
       setType(HBaseProtos.RegionSpecifier.RegionSpecifierType.REGION_NAME).
-      setValue(ByteString.copyFrom(hri.getRegionName())).build()).build());
+      setValue(ByteString.copyFrom(regionInfo.getRegionName())).build()).build());
   }
 %>
 <%
@@ -205,10 +208,10 @@ if ( fqtn != null ) {
     %> Split request accepted. <%
     } else if (action.equals("compact")) {
       if (key != null && key.length() > 0) {
-        List<HRegionInfo> regions = admin.getTableRegions(TableName.valueOf(fqtn));
+        List<RegionInfo> regions = admin.getRegions(TableName.valueOf(fqtn));
         byte[] row = Bytes.toBytes(key);
 
-        for (HRegionInfo region : regions) {
+        for (RegionInfo region : regions) {
           if (region.containsRow(row)) {
             admin.compactRegion(region.getRegionName());
           }
@@ -244,8 +247,8 @@ if ( fqtn != null ) {
 <%
   // NOTE: Presumes meta with one or more replicas
   for (int j = 0; j < numMetaReplicas; j++) {
-    HRegionInfo meta = RegionReplicaUtil.getRegionInfoForReplica(
-                            HRegionInfo.FIRST_META_REGIONINFO, j);
+    RegionInfo meta = RegionReplicaUtil.getRegionInfoForReplica(
+                            RegionInfoBuilder.FIRST_META_REGIONINFO, j);
     ServerName metaLocation = metaTableLocator.waitMetaRegionLocation(master.getZooKeeper(), j, 1);
     for (int i = 0; i < 1; i++) {
       String url = "";
@@ -433,10 +436,10 @@ if ( fqtn != null ) {
   Map<ServerName, Integer> regDistribution = new TreeMap<>();
   Map<ServerName, Integer> primaryRegDistribution = new TreeMap<>();
   List<HRegionLocation> regions = r.getAllRegionLocations();
-  Map<HRegionInfo, RegionLoad> regionsToLoad = new LinkedHashMap<>();
-  Map<HRegionInfo, ServerName> regionsToServer = new LinkedHashMap<>();
+  Map<RegionInfo, RegionLoad> regionsToLoad = new LinkedHashMap<>();
+  Map<RegionInfo, ServerName> regionsToServer = new LinkedHashMap<>();
   for (HRegionLocation hriEntry : regions) {
-    HRegionInfo regionInfo = hriEntry.getRegionInfo();
+    RegionInfo regionInfo = hriEntry.getRegionInfo();
     ServerName addr = hriEntry.getServerName();
     regionsToServer.put(regionInfo, addr);
 
@@ -506,14 +509,14 @@ ShowDetailName&Start/End Key<input type="checkbox" id="showWhole" style="margin-
 </tr>
 
 <%
-  List<Map.Entry<HRegionInfo, RegionLoad>> entryList = new ArrayList<>(regionsToLoad.entrySet());
+  List<Map.Entry<RegionInfo, RegionLoad>> entryList = new ArrayList<>(regionsToLoad.entrySet());
   if(sortKey != null) {
     if (sortKey.equals("readrequest")) {
       Collections.sort(entryList,
-          new Comparator<Map.Entry<HRegionInfo, RegionLoad>>() {
+          new Comparator<Map.Entry<RegionInfo, RegionLoad>>() {
             public int compare(
-                Map.Entry<HRegionInfo, RegionLoad> entry1,
-                Map.Entry<HRegionInfo, RegionLoad> entry2) {
+                Map.Entry<RegionInfo, RegionLoad> entry1,
+                Map.Entry<RegionInfo, RegionLoad> entry2) {
               if (entry1 == null || entry1.getValue() == null) {
                 return -1;
               } else if (entry2 == null || entry2.getValue() == null) {
@@ -533,10 +536,10 @@ ShowDetailName&Start/End Key<input type="checkbox" id="showWhole" style="margin-
           });
     } else if (sortKey.equals("writerequest")) {
       Collections.sort(entryList,
-          new Comparator<Map.Entry<HRegionInfo, RegionLoad>>() {
+          new Comparator<Map.Entry<RegionInfo, RegionLoad>>() {
             public int compare(
-                Map.Entry<HRegionInfo, RegionLoad> entry1,
-                Map.Entry<HRegionInfo, RegionLoad> entry2) {
+                Map.Entry<RegionInfo, RegionLoad> entry1,
+                Map.Entry<RegionInfo, RegionLoad> entry2) {
               if (entry1 == null || entry1.getValue() == null) {
                 return -1;
               } else if (entry2 == null || entry2.getValue() == null) {
@@ -558,10 +561,10 @@ ShowDetailName&Start/End Key<input type="checkbox" id="showWhole" style="margin-
           });
     } else if (sortKey.equals("size")) {
       Collections.sort(entryList,
-          new Comparator<Map.Entry<HRegionInfo, RegionLoad>>() {
+          new Comparator<Map.Entry<RegionInfo, RegionLoad>>() {
             public int compare(
-                Map.Entry<HRegionInfo, RegionLoad> entry1,
-                Map.Entry<HRegionInfo, RegionLoad> entry2) {
+                Map.Entry<RegionInfo, RegionLoad> entry1,
+                Map.Entry<RegionInfo, RegionLoad> entry2) {
               if (entry1 == null || entry1.getValue() == null) {
                 return -1;
               } else if (entry2 == null || entry2.getValue() == null) {
@@ -583,10 +586,10 @@ ShowDetailName&Start/End Key<input type="checkbox" id="showWhole" style="margin-
           });
     } else if (sortKey.equals("filecount")) {
       Collections.sort(entryList,
-          new Comparator<Map.Entry<HRegionInfo, RegionLoad>>() {
+          new Comparator<Map.Entry<RegionInfo, RegionLoad>>() {
             public int compare(
-                Map.Entry<HRegionInfo, RegionLoad> entry1,
-                Map.Entry<HRegionInfo, RegionLoad> entry2) {
+                Map.Entry<RegionInfo, RegionLoad> entry1,
+                Map.Entry<RegionInfo, RegionLoad> entry2) {
               if (entry1 == null || entry1.getValue() == null) {
                 return -1;
               } else if (entry2 == null || entry2.getValue() == null) {
@@ -608,10 +611,10 @@ ShowDetailName&Start/End Key<input type="checkbox" id="showWhole" style="margin-
           });
     } else if (sortKey.equals("memstore")) {
       Collections.sort(entryList,
-          new Comparator<Map.Entry<HRegionInfo, RegionLoad>>() {
+          new Comparator<Map.Entry<RegionInfo, RegionLoad>>() {
             public int compare(
-                Map.Entry<HRegionInfo, RegionLoad> entry1,
-                Map.Entry<HRegionInfo, RegionLoad> entry2) {
+                Map.Entry<RegionInfo, RegionLoad> entry1,
+                Map.Entry<RegionInfo, RegionLoad> entry2) {
               if (entry1 == null || entry1.getValue()==null) {
                 return -1;
               } else if (entry2 == null || entry2.getValue()==null) {
@@ -633,10 +636,10 @@ ShowDetailName&Start/End Key<input type="checkbox" id="showWhole" style="margin-
           });
     } else if (sortKey.equals("locality")) {
       Collections.sort(entryList,
-          new Comparator<Map.Entry<HRegionInfo, RegionLoad>>() {
+          new Comparator<Map.Entry<RegionInfo, RegionLoad>>() {
             public int compare(
-                Map.Entry<HRegionInfo, RegionLoad> entry1,
-                Map.Entry<HRegionInfo, RegionLoad> entry2) {
+                Map.Entry<RegionInfo, RegionLoad> entry1,
+                Map.Entry<RegionInfo, RegionLoad> entry2) {
               if (entry1 == null || entry1.getValue()==null) {
                 return -1;
               } else if (entry2 == null || entry2.getValue()==null) {
@@ -664,8 +667,8 @@ ShowDetailName&Start/End Key<input type="checkbox" id="showWhole" style="margin-
   if (numRegionsToRender < 0) {
     numRegionsToRender = numRegions;
   }
-  for (Map.Entry<HRegionInfo, RegionLoad> hriEntry : entryList) {
-    HRegionInfo regionInfo = hriEntry.getKey();
+  for (Map.Entry<RegionInfo, RegionLoad> hriEntry : entryList) {
+    RegionInfo regionInfo = hriEntry.getKey();
     ServerName addr = regionsToServer.get(regionInfo);
     RegionLoad load = hriEntry.getValue();
     String readReq = "N/A";

http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/main/resources/hbase-webapps/regionserver/region.jsp
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/resources/hbase-webapps/regionserver/region.jsp b/hbase-server/src/main/resources/hbase-webapps/regionserver/region.jsp
index 097b8a9..62c589b 100644
--- a/hbase-server/src/main/resources/hbase-webapps/regionserver/region.jsp
+++ b/hbase-server/src/main/resources/hbase-webapps/regionserver/region.jsp
@@ -23,21 +23,20 @@
   import="java.util.List"
   import="static org.apache.commons.lang3.StringEscapeUtils.escapeXml"
   import="org.apache.hadoop.conf.Configuration"
-  import="org.apache.hadoop.hbase.HTableDescriptor"
-  import="org.apache.hadoop.hbase.HColumnDescriptor"
   import="org.apache.hadoop.hbase.HBaseConfiguration"
-  import="org.apache.hadoop.hbase.HRegionInfo"
+  import="org.apache.hadoop.hbase.client.RegionInfoDisplay"
   import="org.apache.hadoop.hbase.regionserver.HRegionServer"
   import="org.apache.hadoop.hbase.regionserver.Region"
   import="org.apache.hadoop.hbase.regionserver.Store"
-  import="org.apache.hadoop.hbase.regionserver.StoreFile"%>
+  import="org.apache.hadoop.hbase.regionserver.StoreFile"
+%>
 <%
   String regionName = request.getParameter("name");
   HRegionServer rs = (HRegionServer) getServletContext().getAttribute(HRegionServer.REGIONSERVER);
   Configuration conf = rs.getConfiguration();
 
   Region region = rs.getRegion(regionName);
-  String displayName = HRegionInfo.getRegionNameAsStringForDisplay(region.getRegionInfo(),
+  String displayName = RegionInfoDisplay.getRegionNameAsStringForDisplay(region.getRegionInfo(),
     rs.getConfiguration());
 %>
 <!DOCTYPE html>

http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
index 6e30246..59ad6de 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
@@ -49,6 +49,7 @@ import java.util.TreeSet;
 import java.util.UUID;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
 
 import edu.umd.cs.findbugs.annotations.Nullable;
 import org.apache.commons.io.FileUtils;
@@ -63,6 +64,9 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.ClusterStatus.Option;
 import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
 import org.apache.hadoop.hbase.Waiter.Predicate;
+import org.apache.hadoop.hbase.client.ImmutableHRegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.BufferedMutator;
@@ -470,7 +474,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
   /**
    * @return META table descriptor
    * @deprecated since 2.0 version and will be removed in 3.0 version.
-   *             use {@link #getMetaDescriptor()}
+   *             use {@link #getMetaTableDescriptorBuilder()}
    */
   @Deprecated
   public HTableDescriptor getMetaTableDescriptor() {
@@ -1468,22 +1472,6 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
    * @param c Configuration to use
    * @return A Table instance for the created table.
    * @throws IOException
-   * @deprecated since 2.0 version and will be removed in 3.0 version.
-   *             use {@link #createTable(TableDescriptor, byte[][], Configuration)}
-   */
-  @Deprecated
-  public Table createTable(HTableDescriptor htd, byte[][] families, Configuration c)
-  throws IOException {
-    return createTable((TableDescriptor) htd, families, c);
-  }
-
-  /**
-   * Create a table.
-   * @param htd
-   * @param families
-   * @param c Configuration to use
-   * @return A Table instance for the created table.
-   * @throws IOException
    */
   public Table createTable(TableDescriptor htd, byte[][] families, Configuration c)
   throws IOException {
@@ -1498,23 +1486,6 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
    * @param c Configuration to use
    * @return A Table instance for the created table.
    * @throws IOException
-   * @deprecated since 2.0 version and will be removed in 3.0 version.
-   *             use {@link #createTable(TableDescriptor, byte[][], byte[][], Configuration)}
-   */
-  @Deprecated
-  public Table createTable(HTableDescriptor htd, byte[][] families, byte[][] splitKeys,
-      Configuration c) throws IOException {
-    return createTable((TableDescriptor) htd, families, splitKeys, c);
-  }
-
-  /**
-   * Create a table.
-   * @param htd
-   * @param families
-   * @param splitKeys
-   * @param c Configuration to use
-   * @return A Table instance for the created table.
-   * @throws IOException
    */
   public Table createTable(TableDescriptor htd, byte[][] families, byte[][] splitKeys,
       Configuration c) throws IOException {
@@ -1541,21 +1512,6 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
    * @param splitRows
    * @return A Table instance for the created table.
    * @throws IOException
-   * @deprecated since 2.0 version and will be removed in 3.0 version.
-   *             use {@link #createTable(TableDescriptor, byte[][])}
-   */
-  @Deprecated
-  public Table createTable(HTableDescriptor htd, byte[][] splitRows)
-      throws IOException {
-    return createTable((TableDescriptor) htd, splitRows);
-  }
-
-  /**
-   * Create a table.
-   * @param htd
-   * @param splitRows
-   * @return A Table instance for the created table.
-   * @throws IOException
    */
   public Table createTable(TableDescriptor htd, byte[][] splitRows)
       throws IOException {
@@ -1745,17 +1701,6 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
 
   /**
    * Modify a table, synchronous. Waiting logic similar to that of {@code admin.rb#alter_status}.
-   * @deprecated since 2.0 version and will be removed in 3.0 version.
-   *             use {@link #modifyTableSync(Admin, TableDescriptor)}
-   */
-  @Deprecated
-  @SuppressWarnings("serial")
-  public static void modifyTableSync(Admin admin, HTableDescriptor desc)
-      throws IOException, InterruptedException {
-    modifyTableSync(admin, (TableDescriptor) desc);
-  }
-  /**
-   * Modify a table, synchronous. Waiting logic similar to that of {@code admin.rb#alter_status}.
    */
   @SuppressWarnings("serial")
   public static void modifyTableSync(Admin admin, TableDescriptor desc)
@@ -1900,22 +1845,6 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
    * @param endKey
    * @return
    * @throws IOException
-   * @deprecated since 2.0 version and will be removed in 3.0 version.
-   *             use {@link #createLocalHRegion(TableDescriptor, byte[], byte[])}
-   */
-  @Deprecated
-  public HRegion createLocalHRegion(HTableDescriptor desc, byte [] startKey,
-      byte [] endKey) throws IOException {
-    return createLocalHRegion((TableDescriptor) desc, startKey, endKey);
-  }
-
-  /**
-   * Create an HRegion that writes to the local tmp dirs
-   * @param desc
-   * @param startKey
-   * @param endKey
-   * @return
-   * @throws IOException
    */
   public HRegion createLocalHRegion(TableDescriptor desc, byte [] startKey,
       byte [] endKey)
@@ -1927,19 +1856,8 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
   /**
    * Create an HRegion that writes to the local tmp dirs. Creates the WAL for you. Be sure to call
    * {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} when you're finished with it.
-   * @deprecated since 2.0 version and will be removed in 3.0 version.
-   *             use {@link #createLocalHRegion(HRegionInfo, TableDescriptor)}
    */
-  @Deprecated
-  public HRegion createLocalHRegion(HRegionInfo info, HTableDescriptor desc) throws IOException {
-    return createLocalHRegion(info, (TableDescriptor) desc);
-  }
-
-  /**
-   * Create an HRegion that writes to the local tmp dirs. Creates the WAL for you. Be sure to call
-   * {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} when you're finished with it.
-   */
-  public HRegion createLocalHRegion(HRegionInfo info, TableDescriptor desc) throws IOException {
+  public HRegion createLocalHRegion(RegionInfo info, TableDescriptor desc) throws IOException {
     return createRegionAndWAL(info, getDataTestDir(), getConfiguration(), desc);
   }
 
@@ -1950,24 +1868,8 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
    * @param wal wal for this region.
    * @return created hregion
    * @throws IOException
-   * @deprecated since 2.0 version and will be removed in 3.0 version.
-   *             use {@link #createLocalHRegion(HRegionInfo, TableDescriptor, WAL)}
-   */
-  @Deprecated
-  public HRegion createLocalHRegion(HRegionInfo info, HTableDescriptor desc, WAL wal)
-      throws IOException {
-    return createLocalHRegion(info, (TableDescriptor) desc, wal);
-  }
-
-  /**
-   * Create an HRegion that writes to the local tmp dirs with specified wal
-   * @param info regioninfo
-   * @param desc table descriptor
-   * @param wal wal for this region.
-   * @return created hregion
-   * @throws IOException
    */
-  public HRegion createLocalHRegion(HRegionInfo info, TableDescriptor desc, WAL wal)
+  public HRegion createLocalHRegion(RegionInfo info, TableDescriptor desc, WAL wal)
       throws IOException {
     return HRegion.createHRegion(info, getDataTestDir(), getConfiguration(), desc, wal);
   }
@@ -2482,7 +2384,8 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
   @Deprecated
   public List<HRegionInfo> createMultiRegionsInMeta(final Configuration conf,
       final HTableDescriptor htd, byte [][] startKeys) throws IOException {
-    return createMultiRegionsInMeta(conf, (TableDescriptor) htd, startKeys);
+    return createMultiRegionsInMeta(conf, (TableDescriptor) htd, startKeys)
+        .stream().map(ImmutableHRegionInfo::new).collect(Collectors.toList());
   }
   /**
    * Create rows in hbase:meta for regions of the specified table with the specified
@@ -2494,19 +2397,21 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
    * @return list of region info for regions added to meta
    * @throws IOException
    */
-  public List<HRegionInfo> createMultiRegionsInMeta(final Configuration conf,
+  public List<RegionInfo> createMultiRegionsInMeta(final Configuration conf,
       final TableDescriptor htd, byte [][] startKeys)
   throws IOException {
     Table meta = getConnection().getTable(TableName.META_TABLE_NAME);
     Arrays.sort(startKeys, Bytes.BYTES_COMPARATOR);
-    List<HRegionInfo> newRegions = new ArrayList<>(startKeys.length);
+    List<RegionInfo> newRegions = new ArrayList<>(startKeys.length);
     MetaTableAccessor
         .updateTableState(getConnection(), htd.getTableName(), TableState.State.ENABLED);
     // add custom ones
     for (int i = 0; i < startKeys.length; i++) {
       int j = (i + 1) % startKeys.length;
-      HRegionInfo hri = new HRegionInfo(htd.getTableName(), startKeys[i],
-          startKeys[j]);
+      RegionInfo hri = RegionInfoBuilder.newBuilder(htd.getTableName())
+          .setStartKey(startKeys[i])
+          .setEndKey(startKeys[j])
+          .build();
       MetaTableAccessor.addRegionToMeta(meta, hri);
       newRegions.add(hri);
     }
@@ -2518,7 +2423,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
   /**
    * Create an unmanaged WAL. Be sure to close it when you're through.
    */
-  public static WAL createWal(final Configuration conf, final Path rootDir, final HRegionInfo hri)
+  public static WAL createWal(final Configuration conf, final Path rootDir, final RegionInfo hri)
       throws IOException {
     // The WAL subsystem will use the default rootDir rather than the passed in rootDir
     // unless I pass along via the conf.
@@ -2533,20 +2438,8 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
   /**
    * Create a region with it's own WAL. Be sure to call
    * {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} to clean up all resources.
-   * @deprecated since 2.0 version and will be removed in 3.0 version.
-   *             use {@link #createRegionAndWAL(HRegionInfo, Path, Configuration, TableDescriptor)}
-   */
-  @Deprecated
-  public static HRegion createRegionAndWAL(final HRegionInfo info, final Path rootDir,
-      final Configuration conf, final HTableDescriptor htd) throws IOException {
-    return createRegionAndWAL(info, rootDir, conf, (TableDescriptor) htd);
-  }
-
-  /**
-   * Create a region with it's own WAL. Be sure to call
-   * {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} to clean up all resources.
    */
-  public static HRegion createRegionAndWAL(final HRegionInfo info, final Path rootDir,
+  public static HRegion createRegionAndWAL(final RegionInfo info, final Path rootDir,
       final Configuration conf, final TableDescriptor htd) throws IOException {
     return createRegionAndWAL(info, rootDir, conf, htd, true);
   }
@@ -2554,21 +2447,8 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
   /**
    * Create a region with it's own WAL. Be sure to call
    * {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} to clean up all resources.
-   * @deprecated since 2.0 version and will be removed in 3.0 version.
-   *             use {@link #createRegionAndWAL(HRegionInfo, Path, Configuration, TableDescriptor, boolean)}
    */
-  @Deprecated
-  public static HRegion createRegionAndWAL(final HRegionInfo info, final Path rootDir,
-      final Configuration conf, final HTableDescriptor htd, boolean initialize)
-      throws IOException {
-    return createRegionAndWAL(info, rootDir, conf, (TableDescriptor) htd, initialize);
-  }
-
-  /**
-   * Create a region with it's own WAL. Be sure to call
-   * {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} to clean up all resources.
-   */
-  public static HRegion createRegionAndWAL(final HRegionInfo info, final Path rootDir,
+  public static HRegion createRegionAndWAL(final RegionInfo info, final Path rootDir,
       final Configuration conf, final TableDescriptor htd, boolean initialize)
       throws IOException {
     ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
@@ -2607,7 +2487,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
     List<byte[]> rows = new ArrayList<>();
     ResultScanner s = t.getScanner(new Scan());
     for (Result result : s) {
-      HRegionInfo info = MetaTableAccessor.getHRegionInfo(result);
+      RegionInfo info = MetaTableAccessor.getRegionInfo(result);
       if (info == null) {
         LOG.error("No region info for row " + Bytes.toString(result.getRow()));
         // TODO figure out what to do for this new hosed case.
@@ -3175,13 +3055,13 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
   public String explainTableAvailability(TableName tableName) throws IOException {
     String msg = explainTableState(tableName, TableState.State.ENABLED) + ", ";
     if (getHBaseCluster().getMaster().isAlive()) {
-      Map<HRegionInfo, ServerName> assignments =
+      Map<RegionInfo, ServerName> assignments =
           getHBaseCluster().getMaster().getAssignmentManager().getRegionStates()
               .getRegionAssignments();
-      final List<Pair<HRegionInfo, ServerName>> metaLocations =
+      final List<Pair<RegionInfo, ServerName>> metaLocations =
           MetaTableAccessor.getTableRegionsAndLocations(connection, tableName);
-      for (Pair<HRegionInfo, ServerName> metaLocation : metaLocations) {
-        HRegionInfo hri = metaLocation.getFirst();
+      for (Pair<RegionInfo, ServerName> metaLocation : metaLocations) {
+        RegionInfo hri = metaLocation.getFirst();
         ServerName sn = metaLocation.getSecond();
         if (!assignments.containsKey(hri)) {
           msg += ", region " + hri
@@ -3377,7 +3257,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
     NavigableSet<String> online = new TreeSet<>();
     for (RegionServerThread rst : cluster.getLiveRegionServerThreads()) {
       try {
-        for (HRegionInfo region :
+        for (RegionInfo region :
             ProtobufUtil.getOnlineRegions(rst.getRegionServer().getRSRpcServices())) {
           online.add(region.getRegionNameAsString());
         }
@@ -3387,7 +3267,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
     }
     for (MasterThread mt : cluster.getLiveMasterThreads()) {
       try {
-        for (HRegionInfo region :
+        for (RegionInfo region :
             ProtobufUtil.getOnlineRegions(mt.getMaster().getRSRpcServices())) {
           online.add(region.getRegionNameAsString());
         }
@@ -3444,7 +3324,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
    * @throw InterruptedException
    * @return true if the region is assigned false otherwise.
    */
-  public boolean assignRegion(final HRegionInfo regionInfo)
+  public boolean assignRegion(final RegionInfo regionInfo)
       throws IOException, InterruptedException {
     final AssignmentManager am = getHBaseCluster().getMaster().getAssignmentManager();
     am.assign(regionInfo);
@@ -3459,7 +3339,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
    * @throws InterruptedException
    * @throws IOException
    */
-  public void moveRegionAndWait(HRegionInfo destRegion, ServerName destServer)
+  public void moveRegionAndWait(RegionInfo destRegion, ServerName destServer)
       throws InterruptedException, IOException {
     HMaster master = getMiniHBaseCluster().getMaster();
     // TODO: Here we start the move. The move can take a while.
@@ -3559,7 +3439,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
 
         @Override
         public boolean evaluate() throws IOException {
-          List<HRegionInfo> hris = states.getRegionsOfTable(tableName);
+          List<RegionInfo> hris = states.getRegionsOfTable(tableName);
           return hris != null && !hris.isEmpty();
         }
       });
@@ -3941,21 +3821,6 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
    * Creates a pre-split table for load testing. If the table already exists,
    * logs a warning and continues.
    * @return the number of regions the table was split into
-   * @deprecated since 2.0 version and will be removed in 3.0 version.
-   *             use {@link #createPreSplitLoadTestTable(Configuration,
-   *                 TableDescriptor, ColumnFamilyDescriptor)}
-   */
-  @Deprecated
-  public static int createPreSplitLoadTestTable(Configuration conf,
-      HTableDescriptor desc, HColumnDescriptor hcd) throws IOException {
-    return createPreSplitLoadTestTable(conf, (TableDescriptor) desc,
-            (ColumnFamilyDescriptor) hcd);
-  }
-
-  /**
-   * Creates a pre-split table for load testing. If the table already exists,
-   * logs a warning and continues.
-   * @return the number of regions the table was split into
    */
   public static int createPreSplitLoadTestTable(Configuration conf,
       TableDescriptor desc, ColumnFamilyDescriptor hcd) throws IOException {
@@ -3966,21 +3831,6 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
    * Creates a pre-split table for load testing. If the table already exists,
    * logs a warning and continues.
    * @return the number of regions the table was split into
-   * @deprecated since 2.0 version and will be removed in 3.0 version.
-   *             use {@link #createPreSplitLoadTestTable(Configuration,
-   *                 TableDescriptor, ColumnFamilyDescriptor, int)}
-   */
-  @Deprecated
-  public static int createPreSplitLoadTestTable(Configuration conf,
-      HTableDescriptor desc, HColumnDescriptor hcd, int numRegionsPerServer) throws IOException {
-    return createPreSplitLoadTestTable(conf, (TableDescriptor) desc,
-            (ColumnFamilyDescriptor) hcd, numRegionsPerServer);
-  }
-
-  /**
-   * Creates a pre-split table for load testing. If the table already exists,
-   * logs a warning and continues.
-   * @return the number of regions the table was split into
    */
   public static int createPreSplitLoadTestTable(Configuration conf,
       TableDescriptor desc, ColumnFamilyDescriptor hcd, int numRegionsPerServer) throws IOException {
@@ -3992,22 +3842,6 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
    * Creates a pre-split table for load testing. If the table already exists,
    * logs a warning and continues.
    * @return the number of regions the table was split into
-   * @deprecated since 2.0 version and will be removed in 3.0 version.
-   *             use {@link #createPreSplitLoadTestTable(Configuration,
-   *                 TableDescriptor, ColumnFamilyDescriptor[], int)}
-   */
-  @Deprecated
-  public static int createPreSplitLoadTestTable(Configuration conf,
-      HTableDescriptor desc, HColumnDescriptor[] hcds,
-      int numRegionsPerServer) throws IOException {
-    return createPreSplitLoadTestTable(conf, (TableDescriptor) desc,
-            (ColumnFamilyDescriptor[]) hcds, numRegionsPerServer);
-  }
-
-  /**
-   * Creates a pre-split table for load testing. If the table already exists,
-   * logs a warning and continues.
-   * @return the number of regions the table was split into
    */
   public static int createPreSplitLoadTestTable(Configuration conf,
       TableDescriptor desc, ColumnFamilyDescriptor[] hcds,
@@ -4081,12 +3915,12 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
    *  is recorded in master.
    */
   public void assertRegionOnServer(
-      final HRegionInfo hri, final ServerName server,
+      final RegionInfo hri, final ServerName server,
       final long timeout) throws IOException, InterruptedException {
     long timeoutTime = System.currentTimeMillis() + timeout;
     while (true) {
-      List<HRegionInfo> regions = getAdmin().getOnlineRegions(server);
-      if (regions.contains(hri)) return;
+      List<RegionInfo> regions = getAdmin().getRegions(server);
+      if (regions.stream().anyMatch(r -> RegionInfo.COMPARATOR.compare(r, hri) == 0)) return;
       long now = System.currentTimeMillis();
       if (now > timeoutTime) break;
       Thread.sleep(10);
@@ -4100,12 +3934,12 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
    * region server, but not on any other one.
    */
   public void assertRegionOnlyOnServer(
-      final HRegionInfo hri, final ServerName server,
+      final RegionInfo hri, final ServerName server,
       final long timeout) throws IOException, InterruptedException {
     long timeoutTime = System.currentTimeMillis() + timeout;
     while (true) {
-      List<HRegionInfo> regions = getAdmin().getOnlineRegions(server);
-      if (regions.contains(hri)) {
+      List<RegionInfo> regions = getAdmin().getRegions(server);
+      if (regions.stream().anyMatch(r -> RegionInfo.COMPARATOR.compare(r, hri) == 0)) {
         List<JVMClusterUtil.RegionServerThread> rsThreads =
           getHBaseCluster().getLiveRegionServerThreads();
         for (JVMClusterUtil.RegionServerThread rsThread: rsThreads) {
@@ -4129,16 +3963,6 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
       + " on server " + server);
   }
 
-  /**
-   * @deprecated since 2.0 version and will be removed in 3.0 version.
-   *             use {@link #createTestRegion(String, ColumnFamilyDescriptor)}
-   */
-  @Deprecated
-  public HRegion createTestRegion(String tableName, HColumnDescriptor cd)
-      throws IOException {
-    return createTestRegion(tableName, (ColumnFamilyDescriptor) cd);
-  }
-
   public HRegion createTestRegion(String tableName, ColumnFamilyDescriptor cd)
       throws IOException {
     TableDescriptor td

http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
index ef398ff..e453be2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
@@ -17,8 +17,6 @@
  */
 package org.apache.hadoop.hbase;
 
-import com.google.protobuf.Service;
-
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.util.Collections;
@@ -34,6 +32,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.locking.EntityLock;
 import org.apache.hadoop.hbase.executor.ExecutorService;
 import org.apache.hadoop.hbase.fs.HFileSystem;
@@ -50,14 +49,17 @@ import org.apache.hadoop.hbase.regionserver.RegionServerServices;
 import org.apache.hadoop.hbase.regionserver.SecureBulkLoadManager;
 import org.apache.hadoop.hbase.regionserver.ServerNonceManager;
 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.zookeeper.KeeperException;
 
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
+
+import com.google.protobuf.Service;
+
 /**
  * Basic mock region server services.  Should only be instantiated by HBaseTestingUtility.b
  */
@@ -242,7 +244,7 @@ public class MockRegionServerServices implements RegionServerServices {
   }
 
   @Override
-  public WAL getWAL(HRegionInfo regionInfo) throws IOException {
+  public WAL getWAL(RegionInfo regionInfo) throws IOException {
     return null;
   }
 
@@ -280,13 +282,13 @@ public class MockRegionServerServices implements RegionServerServices {
 
   @Override
   public boolean reportRegionStateTransition(TransitionCode code, long openSeqNum,
-      HRegionInfo... hris) {
+      RegionInfo... hris) {
     return false;
   }
 
   @Override
   public boolean reportRegionStateTransition(TransitionCode code,
-      HRegionInfo... hris) {
+      RegionInfo... hris) {
     return false;
   }
 
@@ -332,7 +334,7 @@ public class MockRegionServerServices implements RegionServerServices {
   }
 
   @Override
-  public EntityLock regionLock(List<HRegionInfo> regionInfos, String description, Abortable abort)
+  public EntityLock regionLock(List<RegionInfo> regionInfos, String description, Abortable abort)
       throws IOException {
     return null;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/TestGlobalMemStoreSize.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestGlobalMemStoreSize.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestGlobalMemStoreSize.java
index e8ae7d1..21b36b7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestGlobalMemStoreSize.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestGlobalMemStoreSize.java
@@ -28,11 +28,11 @@ import java.util.List;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.Region;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -43,6 +43,8 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+
 /**
  * Test HBASE-3694 whether the GlobalMemStoreSize is the same as the summary
  * of all the online region's MemStoreSize
@@ -91,7 +93,7 @@ public class TestGlobalMemStoreSize {
 
     for (HRegionServer server : getOnlineRegionServers()) {
       long globalMemStoreSize = 0;
-      for (HRegionInfo regionInfo :
+      for (RegionInfo regionInfo :
           ProtobufUtil.getOnlineRegions(null, server.getRSRpcServices())) {
         globalMemStoreSize += server.getRegion(regionInfo.getEncodedName()).getMemstoreSize();
       }
@@ -105,7 +107,7 @@ public class TestGlobalMemStoreSize {
       LOG.info("Starting flushes on " + server.getServerName() +
         ", size=" + server.getRegionServerAccounting().getGlobalMemstoreDataSize());
 
-      for (HRegionInfo regionInfo :
+      for (RegionInfo regionInfo :
           ProtobufUtil.getOnlineRegions(null, server.getRSRpcServices())) {
         Region r = server.getRegion(regionInfo.getEncodedName());
         flush(r, server);
@@ -121,7 +123,7 @@ public class TestGlobalMemStoreSize {
       if (size > 0) {
         // If size > 0, see if its because the meta region got edits while
         // our test was running....
-        for (HRegionInfo regionInfo :
+        for (RegionInfo regionInfo :
             ProtobufUtil.getOnlineRegions(null, server.getRSRpcServices())) {
           Region r = server.getRegion(regionInfo.getEncodedName());
           long l = r.getMemstoreSize();

http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java
index c9881c6..b759261 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.regionserver.CompactingMemStore;
@@ -99,7 +100,7 @@ public class TestIOFencing {
 
     @SuppressWarnings("deprecation")
     public CompactionBlockerRegion(Path tableDir, WAL log,
-        FileSystem fs, Configuration confParam, HRegionInfo info,
+        FileSystem fs, Configuration confParam, RegionInfo info,
         TableDescriptor htd, RegionServerServices rsServices) {
       super(tableDir, log, fs, confParam, info, htd, rsServices);
     }
@@ -159,7 +160,7 @@ public class TestIOFencing {
   public static class BlockCompactionsInPrepRegion extends CompactionBlockerRegion {
 
     public BlockCompactionsInPrepRegion(Path tableDir, WAL log,
-        FileSystem fs, Configuration confParam, HRegionInfo info,
+        FileSystem fs, Configuration confParam, RegionInfo info,
         TableDescriptor htd, RegionServerServices rsServices) {
       super(tableDir, log, fs, confParam, info, htd, rsServices);
     }
@@ -182,7 +183,7 @@ public class TestIOFencing {
    */
   public static class BlockCompactionsInCompletionRegion extends CompactionBlockerRegion {
     public BlockCompactionsInCompletionRegion(Path tableDir, WAL log,
-        FileSystem fs, Configuration confParam, HRegionInfo info,
+        FileSystem fs, Configuration confParam, RegionInfo info,
         TableDescriptor htd, RegionServerServices rsServices) {
       super(tableDir, log, fs, confParam, info, htd, rsServices);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java
index a0cd236..15c0b0c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java
@@ -41,6 +41,8 @@ import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Table;
@@ -63,9 +65,9 @@ import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
 
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
-import org.junit.rules.TestName;
 
 /**
  * Test {@link org.apache.hadoop.hbase.MetaTableAccessor}.
@@ -114,7 +116,7 @@ public class TestMetaTableAccessor {
       regionCount = r.getStartKeys().length;
     }
     // Test it works getting a region from just made user table.
-    final List<HRegionInfo> regions =
+    final List<RegionInfo> regions =
       testGettingTableRegions(connection, tableName, regionCount);
     MetaTask reader = new MetaTask(connection, "reader") {
       @Override
@@ -227,7 +229,7 @@ public class TestMetaTableAccessor {
 
   @Test public void testGetRegionsFromMetaTable()
   throws IOException, InterruptedException {
-    List<HRegionInfo> regions =
+    List<RegionInfo> regions =
       new MetaTableLocator().getMetaRegions(UTIL.getZooKeeperWatcher());
     assertTrue(regions.size() >= 1);
     assertTrue(new MetaTableLocator().getMetaRegionsAndLocations(
@@ -256,7 +258,7 @@ public class TestMetaTableAccessor {
     final String name = this.name.getMethodName();
     LOG.info("Started " + name);
     // Test get on non-existent region.
-    Pair<HRegionInfo, ServerName> pair =
+    Pair<RegionInfo, ServerName> pair =
       MetaTableAccessor.getRegion(connection, Bytes.toBytes("nonexistent-region"));
     assertNull(pair);
     LOG.info("Finished " + name);
@@ -285,12 +287,12 @@ public class TestMetaTableAccessor {
     assertEquals(1, MetaTableAccessor.getTableRegions(connection, greaterName).size());
   }
 
-  private static List<HRegionInfo> testGettingTableRegions(final Connection connection,
+  private static List<RegionInfo> testGettingTableRegions(final Connection connection,
       final TableName name, final int regionCount)
   throws IOException, InterruptedException {
-    List<HRegionInfo> regions = MetaTableAccessor.getTableRegions(connection, name);
+    List<RegionInfo> regions = MetaTableAccessor.getTableRegions(connection, name);
     assertEquals(regionCount, regions.size());
-    Pair<HRegionInfo, ServerName> pair =
+    Pair<RegionInfo, ServerName> pair =
       MetaTableAccessor.getRegion(connection, regions.get(0).getRegionName());
     assertEquals(regions.get(0).getEncodedName(),
       pair.getFirst().getEncodedName());
@@ -298,9 +300,9 @@ public class TestMetaTableAccessor {
   }
 
   private static void testGetRegion(final Connection connection,
-      final HRegionInfo region)
+      final RegionInfo region)
   throws IOException, InterruptedException {
-    Pair<HRegionInfo, ServerName> pair =
+    Pair<RegionInfo, ServerName> pair =
       MetaTableAccessor.getRegion(connection, region.getRegionName());
     assertEquals(region.getEncodedName(),
       pair.getFirst().getEncodedName());
@@ -349,12 +351,27 @@ public class TestMetaTableAccessor {
     ServerName serverName100 = ServerName.valueOf("baz", 60010, random.nextLong());
 
     long regionId = System.currentTimeMillis();
-    HRegionInfo primary = new HRegionInfo(TableName.valueOf(name.getMethodName()),
-      HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false, regionId, 0);
-    HRegionInfo replica1 = new HRegionInfo(TableName.valueOf(name.getMethodName()),
-      HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false, regionId, 1);
-    HRegionInfo replica100 = new HRegionInfo(TableName.valueOf(name.getMethodName()),
-      HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false, regionId, 100);
+    RegionInfo primary = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+        .setStartKey(HConstants.EMPTY_START_ROW)
+        .setEndKey(HConstants.EMPTY_END_ROW)
+        .setSplit(false)
+        .setRegionId(regionId)
+        .setReplicaId(0)
+        .build();
+    RegionInfo replica1 = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+        .setStartKey(HConstants.EMPTY_START_ROW)
+        .setEndKey(HConstants.EMPTY_END_ROW)
+        .setSplit(false)
+        .setRegionId(regionId)
+        .setReplicaId(1)
+        .build();
+    RegionInfo replica100 = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+        .setStartKey(HConstants.EMPTY_START_ROW)
+        .setEndKey(HConstants.EMPTY_END_ROW)
+        .setSplit(false)
+        .setRegionId(regionId)
+        .setReplicaId(100)
+        .build();
 
     long seqNum0 = random.nextLong();
     long seqNum1 = random.nextLong();
@@ -422,12 +439,17 @@ public class TestMetaTableAccessor {
   @Test
   public void testMetaLocationForRegionReplicasIsAddedAtTableCreation() throws IOException {
     long regionId = System.currentTimeMillis();
-    HRegionInfo primary = new HRegionInfo(TableName.valueOf(name.getMethodName()),
-      HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false, regionId, 0);
+    RegionInfo primary = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+        .setStartKey(HConstants.EMPTY_START_ROW)
+        .setEndKey(HConstants.EMPTY_END_ROW)
+        .setSplit(false)
+        .setRegionId(regionId)
+        .setReplicaId(0)
+        .build();
 
     Table meta = MetaTableAccessor.getMetaHTable(connection);
     try {
-      List<HRegionInfo> regionInfos = Lists.newArrayList(primary);
+      List<RegionInfo> regionInfos = Lists.newArrayList(primary);
       MetaTableAccessor.addRegionsToMeta(connection, regionInfos, 3);
 
       assertEmptyMetaLocation(meta, primary.getRegionName(), 1);
@@ -441,17 +463,33 @@ public class TestMetaTableAccessor {
   public void testMetaLocationForRegionReplicasIsAddedAtRegionSplit() throws IOException {
     long regionId = System.currentTimeMillis();
     ServerName serverName0 = ServerName.valueOf("foo", 60010, random.nextLong());
-    HRegionInfo parent = new HRegionInfo(TableName.valueOf(name.getMethodName()),
-      HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false, regionId, 0);
-    HRegionInfo splitA = new HRegionInfo(TableName.valueOf(name.getMethodName()),
-      HConstants.EMPTY_START_ROW, Bytes.toBytes("a"), false, regionId+1, 0);
-    HRegionInfo splitB = new HRegionInfo(TableName.valueOf(name.getMethodName()),
-      Bytes.toBytes("a"), HConstants.EMPTY_END_ROW, false, regionId+1, 0);
+    RegionInfo parent = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+        .setStartKey(HConstants.EMPTY_START_ROW)
+        .setEndKey(HConstants.EMPTY_END_ROW)
+        .setSplit(false)
+        .setRegionId(regionId)
+        .setReplicaId(0)
+        .build();
+
+    RegionInfo splitA = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+        .setStartKey(HConstants.EMPTY_START_ROW)
+        .setEndKey(Bytes.toBytes("a"))
+        .setSplit(false)
+        .setRegionId(regionId + 1)
+        .setReplicaId(0)
+        .build();
+    RegionInfo splitB = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+        .setStartKey(Bytes.toBytes("a"))
+        .setEndKey(HConstants.EMPTY_END_ROW)
+        .setSplit(false)
+        .setRegionId(regionId + 1)
+        .setReplicaId(0)
+        .build();
 
 
     Table meta = MetaTableAccessor.getMetaHTable(connection);
     try {
-      List<HRegionInfo> regionInfos = Lists.newArrayList(parent);
+      List<RegionInfo> regionInfos = Lists.newArrayList(parent);
       MetaTableAccessor.addRegionsToMeta(connection, regionInfos, 3);
 
       MetaTableAccessor.splitRegion(connection, parent, splitA, splitB, serverName0, 3, false);
@@ -470,16 +508,32 @@ public class TestMetaTableAccessor {
     long regionId = System.currentTimeMillis();
     ServerName serverName0 = ServerName.valueOf("foo", 60010, random.nextLong());
 
-    HRegionInfo parentA = new HRegionInfo(TableName.valueOf(name.getMethodName()),
-      Bytes.toBytes("a"), HConstants.EMPTY_END_ROW, false, regionId, 0);
-    HRegionInfo parentB = new HRegionInfo(TableName.valueOf(name.getMethodName()),
-      HConstants.EMPTY_START_ROW, Bytes.toBytes("a"), false, regionId, 0);
-    HRegionInfo merged = new HRegionInfo(TableName.valueOf(name.getMethodName()),
-      HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false, regionId+1, 0);
+    RegionInfo parentA = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+        .setStartKey(Bytes.toBytes("a"))
+        .setEndKey(HConstants.EMPTY_END_ROW)
+        .setSplit(false)
+        .setRegionId(regionId)
+        .setReplicaId(0)
+        .build();
+
+    RegionInfo parentB = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+        .setStartKey(HConstants.EMPTY_START_ROW)
+        .setEndKey(Bytes.toBytes("a"))
+        .setSplit(false)
+        .setRegionId(regionId)
+        .setReplicaId(0)
+        .build();
+    RegionInfo merged = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+        .setStartKey(HConstants.EMPTY_START_ROW)
+        .setEndKey(HConstants.EMPTY_END_ROW)
+        .setSplit(false)
+        .setRegionId(regionId + 1)
+        .setReplicaId(0)
+        .build();
 
     Table meta = MetaTableAccessor.getMetaHTable(connection);
     try {
-      List<HRegionInfo> regionInfos = Lists.newArrayList(parentA, parentB);
+      List<RegionInfo> regionInfos = Lists.newArrayList(parentA, parentB);
       MetaTableAccessor.addRegionsToMeta(connection, regionInfos, 3);
 
       MetaTableAccessor.mergeRegions(connection, merged, parentA, parentB, serverName0, 3,
@@ -542,13 +596,18 @@ public class TestMetaTableAccessor {
   @Test
   public void testMastersSystemTimeIsUsedInUpdateLocations() throws IOException {
     long regionId = System.currentTimeMillis();
-    HRegionInfo regionInfo = new HRegionInfo(TableName.valueOf(name.getMethodName()),
-      HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false, regionId, 0);
+    RegionInfo regionInfo = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+        .setStartKey(HConstants.EMPTY_START_ROW)
+        .setEndKey(HConstants.EMPTY_END_ROW)
+        .setSplit(false)
+        .setRegionId(regionId)
+        .setReplicaId(0)
+        .build();
 
     ServerName sn = ServerName.valueOf("bar", 0, 0);
     Table meta = MetaTableAccessor.getMetaHTable(connection);
     try {
-      List<HRegionInfo> regionInfos = Lists.newArrayList(regionInfo);
+      List<RegionInfo> regionInfos = Lists.newArrayList(regionInfo);
       MetaTableAccessor.addRegionsToMeta(connection, regionInfos, 1);
 
       long masterSystemTime = EnvironmentEdgeManager.currentTime() + 123456789;
@@ -579,17 +638,34 @@ public class TestMetaTableAccessor {
   @Test
   public void testMastersSystemTimeIsUsedInMergeRegions() throws IOException {
     long regionId = System.currentTimeMillis();
-    HRegionInfo regionInfoA = new HRegionInfo(TableName.valueOf(name.getMethodName()),
-      HConstants.EMPTY_START_ROW, new byte[] {'a'}, false, regionId, 0);
-    HRegionInfo regionInfoB = new HRegionInfo(TableName.valueOf(name.getMethodName()),
-      new byte[] {'a'}, HConstants.EMPTY_END_ROW, false, regionId, 0);
-    HRegionInfo mergedRegionInfo = new HRegionInfo(TableName.valueOf(name.getMethodName()),
-      HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false, regionId, 0);
+
+    RegionInfo regionInfoA = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+        .setStartKey(HConstants.EMPTY_START_ROW)
+        .setEndKey(new byte[] {'a'})
+        .setSplit(false)
+        .setRegionId(regionId)
+        .setReplicaId(0)
+        .build();
+
+    RegionInfo regionInfoB = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+        .setStartKey(new byte[] {'a'})
+        .setEndKey(HConstants.EMPTY_END_ROW)
+        .setSplit(false)
+        .setRegionId(regionId)
+        .setReplicaId(0)
+        .build();
+    RegionInfo mergedRegionInfo = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+        .setStartKey(HConstants.EMPTY_START_ROW)
+        .setEndKey(HConstants.EMPTY_END_ROW)
+        .setSplit(false)
+        .setRegionId(regionId)
+        .setReplicaId(0)
+        .build();
 
     ServerName sn = ServerName.valueOf("bar", 0, 0);
     Table meta = MetaTableAccessor.getMetaHTable(connection);
     try {
-      List<HRegionInfo> regionInfos = Lists.newArrayList(regionInfoA, regionInfoB);
+      List<RegionInfo> regionInfos = Lists.newArrayList(regionInfoA, regionInfoB);
       MetaTableAccessor.addRegionsToMeta(connection, regionInfos, 1);
 
       // write the serverName column with a big current time, but set the masters time as even
@@ -674,13 +750,21 @@ public class TestMetaTableAccessor {
       UTIL.createTable(tableName, "cf1");
 
       HRegionLocation loc = rl.getAllRegionLocations().get(0);
-      HRegionInfo parent = loc.getRegionInfo();
+      RegionInfo parent = loc.getRegionInfo();
       long rid = 1000;
       byte[] splitKey = Bytes.toBytes("a");
-      HRegionInfo splitA = new HRegionInfo(parent.getTable(), parent.getStartKey(),
-        splitKey, false, rid);
-      HRegionInfo splitB = new HRegionInfo(parent.getTable(), splitKey,
-        parent.getEndKey(), false, rid);
+      RegionInfo splitA = RegionInfoBuilder.newBuilder(parent.getTable())
+          .setStartKey(parent.getStartKey())
+          .setEndKey(splitKey)
+          .setSplit(false)
+          .setRegionId(rid)
+          .build();
+      RegionInfo splitB = RegionInfoBuilder.newBuilder(parent.getTable())
+          .setStartKey(splitKey)
+          .setEndKey(parent.getEndKey())
+          .setSplit(false)
+          .setRegionId(rid)
+          .build();
 
       // find the meta server
       MiniHBaseCluster cluster = UTIL.getMiniHBaseCluster();
@@ -704,16 +788,31 @@ public class TestMetaTableAccessor {
   public void testEmptyMetaDaughterLocationDuringSplit() throws IOException {
     long regionId = System.currentTimeMillis();
     ServerName serverName0 = ServerName.valueOf("foo", 60010, random.nextLong());
-    HRegionInfo parent = new HRegionInfo(TableName.valueOf("table_foo"), HConstants.EMPTY_START_ROW,
-        HConstants.EMPTY_END_ROW, false, regionId, 0);
-    HRegionInfo splitA = new HRegionInfo(TableName.valueOf("table_foo"), HConstants.EMPTY_START_ROW,
-        Bytes.toBytes("a"), false, regionId + 1, 0);
-    HRegionInfo splitB = new HRegionInfo(TableName.valueOf("table_foo"), Bytes.toBytes("a"),
-        HConstants.EMPTY_END_ROW, false, regionId + 1, 0);
+    RegionInfo parent = RegionInfoBuilder.newBuilder(TableName.valueOf("table_foo"))
+        .setStartKey(HConstants.EMPTY_START_ROW)
+        .setEndKey(HConstants.EMPTY_END_ROW)
+        .setSplit(false)
+        .setRegionId(regionId)
+        .setReplicaId(0)
+        .build();
+    RegionInfo splitA = RegionInfoBuilder.newBuilder(TableName.valueOf("table_foo"))
+        .setStartKey(HConstants.EMPTY_START_ROW)
+        .setEndKey(Bytes.toBytes("a"))
+        .setSplit(false)
+        .setRegionId(regionId + 1)
+        .setReplicaId(0)
+        .build();
+    RegionInfo splitB = RegionInfoBuilder.newBuilder(TableName.valueOf("table_foo"))
+        .setStartKey(Bytes.toBytes("a"))
+        .setEndKey(HConstants.EMPTY_END_ROW)
+        .setSplit(false)
+        .setRegionId(regionId + 1)
+        .setReplicaId(0)
+        .build();
 
     Table meta = MetaTableAccessor.getMetaHTable(connection);
     try {
-      List<HRegionInfo> regionInfos = Lists.newArrayList(parent);
+      List<RegionInfo> regionInfos = Lists.newArrayList(parent);
       MetaTableAccessor.addRegionsToMeta(connection, regionInfos, 3);
 
       MetaTableAccessor.splitRegion(connection, parent, splitA, splitB, serverName0, 3, false);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessorNoCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessorNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessorNoCluster.java
index 9915f99..cec2c20 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessorNoCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessorNoCluster.java
@@ -31,6 +31,8 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.ipc.HBaseRpcController;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
@@ -85,31 +87,30 @@ public class TestMetaTableAccessorNoCluster {
 
   @Test
   public void testGetHRegionInfo() throws IOException {
-    assertNull(MetaTableAccessor.getHRegionInfo(new Result()));
+    assertNull(MetaTableAccessor.getRegionInfo(new Result()));
 
     List<Cell> kvs = new ArrayList<>();
     Result r = Result.create(kvs);
-    assertNull(MetaTableAccessor.getHRegionInfo(r));
+    assertNull(MetaTableAccessor.getRegionInfo(r));
 
     byte [] f = HConstants.CATALOG_FAMILY;
     // Make a key value that doesn't have the expected qualifier.
     kvs.add(new KeyValue(HConstants.EMPTY_BYTE_ARRAY, f,
       HConstants.SERVER_QUALIFIER, f));
     r = Result.create(kvs);
-    assertNull(MetaTableAccessor.getHRegionInfo(r));
+    assertNull(MetaTableAccessor.getRegionInfo(r));
     // Make a key that does not have a regioninfo value.
     kvs.add(new KeyValue(HConstants.EMPTY_BYTE_ARRAY, f,
       HConstants.REGIONINFO_QUALIFIER, f));
-    HRegionInfo hri = MetaTableAccessor.getHRegionInfo(Result.create(kvs));
+    RegionInfo hri = MetaTableAccessor.getRegionInfo(Result.create(kvs));
     assertTrue(hri == null);
     // OK, give it what it expects
     kvs.clear();
     kvs.add(new KeyValue(HConstants.EMPTY_BYTE_ARRAY, f,
-      HConstants.REGIONINFO_QUALIFIER,
-      HRegionInfo.FIRST_META_REGIONINFO.toByteArray()));
-    hri = MetaTableAccessor.getHRegionInfo(Result.create(kvs));
+      HConstants.REGIONINFO_QUALIFIER, RegionInfo.toByteArray(RegionInfoBuilder.FIRST_META_REGIONINFO)));
+    hri = MetaTableAccessor.getRegionInfo(Result.create(kvs));
     assertNotNull(hri);
-    assertTrue(hri.equals(HRegionInfo.FIRST_META_REGIONINFO));
+    assertTrue(RegionInfo.COMPARATOR.compare(hri, RegionInfoBuilder.FIRST_META_REGIONINFO) == 0);
   }
 
   /**
@@ -145,7 +146,7 @@ public class TestMetaTableAccessorNoCluster {
       final byte [] rowToVerify = Bytes.toBytes("rowToVerify");
       kvs.add(new KeyValue(rowToVerify,
         HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
-        HRegionInfo.FIRST_META_REGIONINFO.toByteArray()));
+          RegionInfo.toByteArray(RegionInfoBuilder.FIRST_META_REGIONINFO)));
       kvs.add(new KeyValue(rowToVerify,
         HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER,
         Bytes.toBytes(sn.getHostAndPort())));
@@ -173,11 +174,11 @@ public class TestMetaTableAccessorNoCluster {
       // to shove this in here first so it gets picked up all over; e.g. by
       // HTable.
       connection = HConnectionTestingUtility.getSpiedConnection(UTIL.getConfiguration());
-      
+
       // Fix the location lookup so it 'works' though no network.  First
       // make an 'any location' object.
       final HRegionLocation anyLocation =
-        new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO, sn);
+        new HRegionLocation(RegionInfoBuilder.FIRST_META_REGIONINFO, sn);
       final RegionLocations rl = new RegionLocations(anyLocation);
       // Return the RegionLocations object when locateRegion
       // The ugly format below comes of 'Important gotcha on spying real objects!' from
@@ -191,10 +192,10 @@ public class TestMetaTableAccessorNoCluster {
         when(connection).getClient(Mockito.any(ServerName.class));
 
       // Scan meta for user tables and verify we got back expected answer.
-      NavigableMap<HRegionInfo, Result> hris =
+      NavigableMap<RegionInfo, Result> hris =
         MetaTableAccessor.getServerUserRegions(connection, sn);
       assertEquals(1, hris.size());
-      assertTrue(hris.firstEntry().getKey().equals(HRegionInfo.FIRST_META_REGIONINFO));
+      assertTrue(RegionInfo.COMPARATOR.compare(hris.firstEntry().getKey(), RegionInfoBuilder.FIRST_META_REGIONINFO) == 0);
       assertTrue(Bytes.equals(rowToVerify, hris.firstEntry().getValue().getRow()));
       // Finally verify that scan was called four times -- three times
       // with exception and then on 4th attempt we succeed

http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java
index 20ad023..d23b4e4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java
@@ -35,6 +35,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.client.ClientScanner;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
@@ -817,11 +818,11 @@ public class TestPartialResultsFromClientSide {
   }
 
   private void moveRegion(Table table, int index) throws IOException{
-    List<Pair<HRegionInfo, ServerName>> regions = MetaTableAccessor
+    List<Pair<RegionInfo, ServerName>> regions = MetaTableAccessor
         .getTableRegionsAndLocations(TEST_UTIL.getConnection(),
             table.getName());
     assertEquals(1, regions.size());
-    HRegionInfo regionInfo = regions.get(0).getFirst();
+    RegionInfo regionInfo = regions.get(0).getFirst();
     ServerName name = TEST_UTIL.getHBaseCluster().getRegionServer(index).getServerName();
     TEST_UTIL.getAdmin().move(regionInfo.getEncodedNameAsBytes(),
         Bytes.toBytes(name.getServerName()));

http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java
index cff1a8d..3f7ea3b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java
@@ -32,9 +32,9 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.testclassification.FlakeyTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -49,6 +49,8 @@ import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
 
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+
 /**
  * Test whether region re-balancing works. (HBASE-71)
  */
@@ -199,7 +201,7 @@ public class TestRegionRebalancing {
         LOG.debug(server.getServerName() + " Avg: " + avg + " actual: " + serverLoad);
         if (!(avg > 2.0 && serverLoad <= avgLoadPlusSlop
             && serverLoad >= avgLoadMinusSlop)) {
-          for (HRegionInfo hri :
+          for (RegionInfo hri :
               ProtobufUtil.getOnlineRegions(server.getRSRpcServices())) {
             if (hri.isMetaRegion()) serverLoad--;
             // LOG.debug(hri.getRegionNameAsString());

http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/TestStochasticBalancerJmxMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestStochasticBalancerJmxMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestStochasticBalancerJmxMetrics.java
index 7ad99c5..f987ea7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestStochasticBalancerJmxMetrics.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestStochasticBalancerJmxMetrics.java
@@ -20,6 +20,13 @@ package org.apache.hadoop.hbase;
 
 import static org.junit.Assert.assertTrue;
 
+import javax.management.MBeanAttributeInfo;
+import javax.management.MBeanInfo;
+import javax.management.MBeanServerConnection;
+import javax.management.ObjectInstance;
+import javax.management.ObjectName;
+import javax.management.remote.JMXConnector;
+import javax.management.remote.JMXConnectorFactory;
 import java.io.IOException;
 import java.util.HashSet;
 import java.util.Hashtable;
@@ -29,17 +36,10 @@ import java.util.Map;
 import java.util.Random;
 import java.util.Set;
 
-import javax.management.MBeanAttributeInfo;
-import javax.management.MBeanInfo;
-import javax.management.MBeanServerConnection;
-import javax.management.ObjectInstance;
-import javax.management.ObjectName;
-import javax.management.remote.JMXConnector;
-import javax.management.remote.JMXConnectorFactory;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.master.balancer.BalancerTestBase;
 import org.apache.hadoop.hbase.master.balancer.StochasticLoadBalancer;
@@ -128,7 +128,7 @@ public class TestStochasticBalancerJmxMetrics extends BalancerTestBase {
     loadBalancer.setConf(conf);
 
     TableName tableName = HConstants.ENSEMBLE_TABLE_NAME;
-    Map<ServerName, List<HRegionInfo>> clusterState = mockClusterServers(mockCluster_ensemble);
+    Map<ServerName, List<RegionInfo>> clusterState = mockClusterServers(mockCluster_ensemble);
     loadBalancer.balanceCluster(tableName, clusterState);
 
     String[] tableNames = new String[] { tableName.getNameAsString() };
@@ -164,7 +164,7 @@ public class TestStochasticBalancerJmxMetrics extends BalancerTestBase {
 
     // table 1
     TableName tableName = TableName.valueOf(TABLE_NAME_1);
-    Map<ServerName, List<HRegionInfo>> clusterState = mockClusterServers(mockCluster_pertable_1);
+    Map<ServerName, List<RegionInfo>> clusterState = mockClusterServers(mockCluster_pertable_1);
     loadBalancer.balanceCluster(tableName, clusterState);
 
     // table 2
@@ -204,7 +204,7 @@ public class TestStochasticBalancerJmxMetrics extends BalancerTestBase {
 
   /**
    * Read the attributes from Hadoop->HBase->Master->Balancer in JMX
-   * @throws IOException 
+   * @throws IOException
    */
   private Set<String> readJmxMetrics() throws IOException {
     JMXConnector connector = null;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java
index 461ff8c..4ed8d91 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java
@@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
@@ -576,8 +577,8 @@ public class TestZooKeeper {
     static boolean retainAssignCalled = false;
 
     @Override
-    public Map<ServerName, List<HRegionInfo>> retainAssignment(
-        Map<HRegionInfo, ServerName> regions, List<ServerName> servers) throws HBaseIOException {
+    public Map<ServerName, List<RegionInfo>> retainAssignment(
+        Map<RegionInfo, ServerName> regions, List<ServerName> servers) throws HBaseIOException {
       retainAssignCalled = true;
       return super.retainAssignment(regions, servers);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java
index cfe4d1f..d7d1b3a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java
@@ -83,7 +83,7 @@ public class HConnectionTestingUtility {
    * itself a mock.
    * @param sn ServerName to include in the region location returned by this
    * <code>connection</code>
-   * @param hri HRegionInfo to include in the location returned when
+   * @param hri RegionInfo to include in the location returned when
    * getRegionLocator is called on the mocked connection
    * @return Mock up a connection that returns a {@link Configuration} when
    * {@link ClusterConnection#getConfiguration()} is called, a 'location' when
@@ -99,7 +99,7 @@ public class HConnectionTestingUtility {
   public static ClusterConnection getMockedConnectionAndDecorate(final Configuration conf,
       final AdminProtos.AdminService.BlockingInterface admin,
       final ClientProtos.ClientService.BlockingInterface client,
-      final ServerName sn, final HRegionInfo hri)
+      final ServerName sn, final RegionInfo hri)
   throws IOException {
     ConnectionImplementation c = Mockito.mock(ConnectionImplementation.class);
     Mockito.when(c.getConfiguration()).thenReturn(conf);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
index 9c100a2..0b7da28 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
@@ -38,7 +38,6 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.InvalidFamilyOperationException;
@@ -588,10 +587,10 @@ public class TestAdmin1 {
       expectedRegions) throws IOException {
     int numRS = c.getCurrentNrHRS();
     List<HRegionLocation> regions = regionLocator.getAllRegionLocations();
-    Map<ServerName, List<HRegionInfo>> server2Regions = new HashMap<>();
+    Map<ServerName, List<RegionInfo>> server2Regions = new HashMap<>();
     for (HRegionLocation loc : regions) {
       ServerName server = loc.getServerName();
-      List<HRegionInfo> regs = server2Regions.get(server);
+      List<RegionInfo> regs = server2Regions.get(server);
       if (regs == null) {
         regs = new ArrayList<>();
         server2Regions.put(server, regs);
@@ -607,7 +606,7 @@ public class TestAdmin1 {
     float average = (float) expectedRegions/numRS;
     int min = (int)Math.floor(average);
     int max = (int)Math.ceil(average);
-    for (List<HRegionInfo> regionList : server2Regions.values()) {
+    for (List<RegionInfo> regionList : server2Regions.values()) {
       assertTrue("numRS=" + numRS + ", min=" + min + ", max=" + max +
         ", size=" + regionList.size() + ", tablesOnMaster=" + tablesOnMaster,
       regionList.size() == min || regionList.size() == max);
@@ -690,7 +689,7 @@ public class TestAdmin1 {
 
     List<HRegionLocation> regions;
     Iterator<HRegionLocation> hris;
-    HRegionInfo hri;
+    RegionInfo hri;
     ClusterConnection conn = (ClusterConnection) TEST_UTIL.getConnection();
     try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
       regions = l.getAllRegionLocations();
@@ -1160,7 +1159,7 @@ public class TestAdmin1 {
     puts.add(put);
     ht.put(puts);
     ht.close();
-    List<Pair<HRegionInfo, ServerName>> regions =
+    List<Pair<RegionInfo, ServerName>> regions =
         MetaTableAccessor.getTableRegionsAndLocations(TEST_UTIL.getConnection(), tableName);
     boolean gotException = false;
     // the element at index 1 would be a replica (since the metareader gives us ordered
@@ -1214,7 +1213,7 @@ public class TestAdmin1 {
             nameofRegionsToMerge,
             true,
             HConstants.NO_NONCE,
-            HConstants.NO_NONCE);   
+            HConstants.NO_NONCE);
       ((ClusterConnection) TEST_UTIL.getAdmin().getConnection()).getMaster()
         .mergeTableRegions(null, request);
     } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException m) {
@@ -1357,12 +1356,12 @@ public class TestAdmin1 {
       TEST_UTIL.createTable(td, splitRows);
       TEST_UTIL.waitTableAvailable(tableName);
 
-      List<HRegionInfo> tableRegions;
-      HRegionInfo regionA;
-      HRegionInfo regionB;
+      List<RegionInfo> tableRegions;
+      RegionInfo regionA;
+      RegionInfo regionB;
 
       // merge with full name
-      tableRegions = admin.getTableRegions(tableName);
+      tableRegions = admin.getRegions(tableName);
       assertEquals(3, admin.getTableRegions(tableName).size());
       regionA = tableRegions.get(0);
       regionB = tableRegions.get(1);
@@ -1373,7 +1372,7 @@ public class TestAdmin1 {
       assertEquals(2, admin.getTableRegions(tableName).size());
 
       // merge with encoded name
-      tableRegions = admin.getTableRegions(tableName);
+      tableRegions = admin.getRegions(tableName);
       regionA = tableRegions.get(0);
       regionB = tableRegions.get(1);
       // TODO convert this to version that is synchronous (See HBASE-16668)

http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
index e95f382..30cd8bf 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
@@ -42,7 +42,6 @@ import org.apache.hadoop.hbase.ClusterStatus.Option;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
@@ -55,11 +54,10 @@ import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.UnknownRegionException;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
 import org.apache.hadoop.hbase.constraint.ConstraintException;
-import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.Region;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -69,12 +67,14 @@ import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
+import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
-import org.junit.Ignore;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+
 
 /**
  * Class to test HBaseAdmin.
@@ -316,10 +316,10 @@ public class TestAdmin2 {
     final TableName tableName = TableName.valueOf(name.getMethodName());
     createTableWithDefaultConf(tableName);
 
-    HRegionInfo info = null;
+    RegionInfo info = null;
     HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(tableName);
-    List<HRegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(rs.getRSRpcServices());
-    for (HRegionInfo regionInfo : onlineRegions) {
+    List<RegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(rs.getRSRpcServices());
+    for (RegionInfo regionInfo : onlineRegions) {
       if (!regionInfo.getTable().isSystemTable()) {
         info = regionInfo;
         admin.unassign(regionInfo.getRegionName(), true);
@@ -344,10 +344,10 @@ public class TestAdmin2 {
     byte[] tableName = Bytes.toBytes(name);
     createTableWithDefaultConf(tableName);
 
-    HRegionInfo info = null;
+    RegionInfo info = null;
     HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(TableName.valueOf(tableName));
-    List<HRegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(rs.getRSRpcServices());
-    for (HRegionInfo regionInfo : onlineRegions) {
+    List<RegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(rs.getRSRpcServices());
+    for (RegionInfo regionInfo : onlineRegions) {
       if (!regionInfo.isMetaTable()) {
         if (regionInfo.getRegionNameAsString().contains(name)) {
           info = regionInfo;
@@ -369,10 +369,10 @@ public class TestAdmin2 {
     final TableName tableName = TableName.valueOf(name.getMethodName());
     createTableWithDefaultConf(tableName);
 
-    HRegionInfo info = null;
+    RegionInfo info = null;
     HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(tableName);
-    List<HRegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(rs.getRSRpcServices());
-    for (HRegionInfo regionInfo : onlineRegions) {
+    List<RegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(rs.getRSRpcServices());
+    for (RegionInfo regionInfo : onlineRegions) {
       if (!regionInfo.isMetaTable()) {
         if (regionInfo.getRegionNameAsString().contains("TestHBACloseRegion2")) {
           info = regionInfo;
@@ -436,7 +436,7 @@ public class TestAdmin2 {
     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
     admin.createTable(desc, startKey, endKey, expectedRegions);
 
-    List<HRegionInfo> RegionInfos = admin.getTableRegions(tableName);
+    List<RegionInfo> RegionInfos = admin.getRegions(tableName);
 
     assertEquals("Tried to create " + expectedRegions + " regions " +
         "but only found " + RegionInfos.size(),
@@ -449,8 +449,8 @@ public class TestAdmin2 {
     HMaster master = cluster.getMaster();
     final TableName tableName = TableName.valueOf(name.getMethodName());
     Admin localAdmin = createTable(tableName);
-    List<HRegionInfo> tableRegions = localAdmin.getTableRegions(tableName);
-    HRegionInfo hri = tableRegions.get(0);
+    List<RegionInfo> tableRegions = localAdmin.getRegions(tableName);
+    RegionInfo hri = tableRegions.get(0);
     AssignmentManager am = master.getAssignmentManager();
     ServerName server = am.getRegionStates().getRegionServerOfRegion(hri);
     localAdmin.move(hri.getEncodedNameAsBytes(), Bytes.toBytes(server.getServerName()));
@@ -620,9 +620,9 @@ public class TestAdmin2 {
 
     try (RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
       HRegionLocation regionLocation = locator.getRegionLocation(Bytes.toBytes("mmm"));
-      HRegionInfo region = regionLocation.getRegionInfo();
+      RegionInfo region = regionLocation.getRegionInfo();
       byte[] regionName = region.getRegionName();
-      Pair<HRegionInfo, ServerName> pair = rawAdmin.getRegion(regionName);
+      Pair<RegionInfo, ServerName> pair = rawAdmin.getRegion(regionName);
       assertTrue(Bytes.equals(regionName, pair.getFirst().getRegionName()));
       pair = rawAdmin.getRegion(region.getEncodedNameAsBytes());
       assertTrue(Bytes.equals(regionName, pair.getFirst().getRegionName()));