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:48 UTC

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
index fac3ef1..7b42d3d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
@@ -40,6 +40,7 @@ import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.regex.Pattern;
+import java.util.stream.Collectors;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -69,8 +70,6 @@ import org.apache.hadoop.hbase.TableNotDisabledException;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.UnknownRegionException;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.yetus.audience.InterfaceStability;
 import org.apache.hadoop.hbase.client.replication.ReplicationSerDeHelper;
 import org.apache.hadoop.hbase.client.replication.TableCFs;
 import org.apache.hadoop.hbase.client.security.SecurityCapability;
@@ -86,6 +85,26 @@ import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
+import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
+import org.apache.hadoop.hbase.snapshot.HBaseSnapshotException;
+import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
+import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
+import org.apache.hadoop.hbase.snapshot.UnknownSnapshotException;
+import org.apache.hadoop.hbase.util.Addressing;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
+import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+import org.apache.zookeeper.KeeperException;
+
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
@@ -184,28 +203,10 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRe
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos;
-import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
-import org.apache.hadoop.hbase.snapshot.HBaseSnapshotException;
-import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
-import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
-import org.apache.hadoop.hbase.snapshot.UnknownSnapshotException;
-import org.apache.hadoop.hbase.util.Addressing;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
-import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
-import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
-import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.util.StringUtils;
-import org.apache.zookeeper.KeeperException;
 
-import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
 import com.google.protobuf.Descriptors;
 import com.google.protobuf.Message;
 import com.google.protobuf.RpcController;
-import java.util.stream.Collectors;
 
 /**
  * HBaseAdmin is no longer a client API. It is marked InterfaceAudience.Private indicating that
@@ -399,12 +400,26 @@ public class HBaseAdmin implements Admin {
 
   @Override
   public List<RegionInfo> getRegions(final ServerName sn) throws IOException {
-    return getOnlineRegions(sn).stream().collect(Collectors.toList());
+    AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
+    // TODO: There is no timeout on this controller. Set one!
+    HBaseRpcController controller = rpcControllerFactory.newController();
+    return ProtobufUtil.getOnlineRegions(controller, admin);
   }
 
   @Override
   public List<RegionInfo> getRegions(final TableName tableName) throws IOException {
-    return getTableRegions(tableName).stream().collect(Collectors.toList());
+    ZooKeeperWatcher zookeeper =
+        new ZooKeeperWatcher(conf, ZK_IDENTIFIER_PREFIX + connection.toString(),
+            new ThrowableAbortable());
+    try {
+      if (TableName.META_TABLE_NAME.equals(tableName)) {
+        return new MetaTableLocator().getMetaRegions(zookeeper);
+      } else {
+        return MetaTableAccessor.getTableRegions(connection, tableName, true);
+      }
+    } finally {
+      zookeeper.close();
+    }
   }
 
   private static class AbortProcedureFuture extends ProcedureFuture<Boolean> {
@@ -1164,13 +1179,7 @@ public class HBaseAdmin implements Admin {
   @Deprecated
   @Override
   public List<HRegionInfo> getOnlineRegions(final ServerName sn) throws IOException {
-    AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
-    // TODO: There is no timeout on this controller. Set one!
-    HBaseRpcController controller = rpcControllerFactory.newController();
-    List<HRegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(controller, admin);
-    return onlineRegions == null ? null : onlineRegions.stream()
-            .map(hri -> new ImmutableHRegionInfo(hri))
-            .collect(Collectors.toList());
+    return getRegions(sn).stream().map(ImmutableHRegionInfo::new).collect(Collectors.toList());
   }
 
   @Override
@@ -1185,14 +1194,14 @@ public class HBaseAdmin implements Admin {
 
   @Override
   public void flushRegion(final byte[] regionName) throws IOException {
-    Pair<HRegionInfo, ServerName> regionServerPair = getRegion(regionName);
+    Pair<RegionInfo, ServerName> regionServerPair = getRegion(regionName);
     if (regionServerPair == null) {
       throw new IllegalArgumentException("Unknown regionname: " + Bytes.toStringBinary(regionName));
     }
     if (regionServerPair.getSecond() == null) {
       throw new NoServerForRegionException(Bytes.toStringBinary(regionName));
     }
-    final HRegionInfo hRegionInfo = regionServerPair.getFirst();
+    final RegionInfo hRegionInfo = regionServerPair.getFirst();
     ServerName serverName = regionServerPair.getSecond();
     final AdminService.BlockingInterface admin = this.connection.getAdmin(serverName);
     Callable<Void> callable = new Callable<Void>() {
@@ -1304,13 +1313,13 @@ public class HBaseAdmin implements Admin {
           checkTableExists(tableName);
           zookeeper = new ZooKeeperWatcher(conf, ZK_IDENTIFIER_PREFIX + connection.toString(),
                   new ThrowableAbortable());
-          List<Pair<HRegionInfo, ServerName>> pairs;
+          List<Pair<RegionInfo, ServerName>> pairs;
           if (TableName.META_TABLE_NAME.equals(tableName)) {
             pairs = new MetaTableLocator().getMetaRegionsAndLocations(zookeeper);
           } else {
             pairs = MetaTableAccessor.getTableRegionsAndLocations(connection, tableName);
           }
-          for (Pair<HRegionInfo, ServerName> pair: pairs) {
+          for (Pair<RegionInfo, ServerName> pair: pairs) {
             if (pair.getFirst().isOffline()) continue;
             if (pair.getSecond() == null) continue;
             try {
@@ -1344,7 +1353,7 @@ public class HBaseAdmin implements Admin {
    */
   private void compactRegion(final byte[] regionName, final byte[] columnFamily,final boolean major)
   throws IOException {
-    Pair<HRegionInfo, ServerName> regionServerPair = getRegion(regionName);
+    Pair<RegionInfo, ServerName> regionServerPair = getRegion(regionName);
     if (regionServerPair == null) {
       throw new IllegalArgumentException("Invalid region: " + Bytes.toStringBinary(regionName));
     }
@@ -1354,7 +1363,7 @@ public class HBaseAdmin implements Admin {
     compact(regionServerPair.getSecond(), regionServerPair.getFirst(), major, columnFamily);
   }
 
-  private void compact(final ServerName sn, final HRegionInfo hri,
+  private void compact(final ServerName sn, final RegionInfo hri,
       final boolean major, final byte [] family)
   throws IOException {
     final AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
@@ -1655,7 +1664,7 @@ public class HBaseAdmin implements Admin {
     }
 
     TableName tableName = null;
-    Pair<HRegionInfo, ServerName> pair;
+    Pair<RegionInfo, ServerName> pair;
 
     for(int i = 0; i < nameofRegionsToMerge.length; i++) {
       pair = getRegion(nameofRegionsToMerge[i]);
@@ -1750,7 +1759,7 @@ public class HBaseAdmin implements Admin {
       throws IOException {
     byte[] encodedNameofRegionToSplit = HRegionInfo.isEncodedRegionName(regionName) ?
         regionName : HRegionInfo.encodeRegionName(regionName).getBytes();
-    Pair<HRegionInfo, ServerName> pair = getRegion(regionName);
+    Pair<RegionInfo, ServerName> pair = getRegion(regionName);
     if (pair != null) {
       if (pair.getFirst() != null &&
           pair.getFirst().getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) {
@@ -1762,11 +1771,10 @@ public class HBaseAdmin implements Admin {
               + Bytes.toStringBinary(encodedNameofRegionToSplit));
     }
 
-    HRegionInfo hri = pair.getFirst();
-    return splitRegionAsync(hri, splitPoint);
+    return splitRegionAsync(pair.getFirst(), splitPoint);
   }
 
-  Future<Void> splitRegionAsync(HRegionInfo hri, byte[] splitPoint) throws IOException {
+  Future<Void> splitRegionAsync(RegionInfo hri, byte[] splitPoint) throws IOException {
     TableName tableName = hri.getTable();
     if (hri.getStartKey() != null && splitPoint != null &&
         Bytes.compareTo(hri.getStartKey(), splitPoint) == 0) {
@@ -1824,7 +1832,7 @@ public class HBaseAdmin implements Admin {
       checkTableExists(tableName);
       zookeeper = new ZooKeeperWatcher(conf, ZK_IDENTIFIER_PREFIX + connection.toString(),
         new ThrowableAbortable());
-      List<Pair<HRegionInfo, ServerName>> pairs;
+      List<Pair<RegionInfo, ServerName>> pairs;
       if (TableName.META_TABLE_NAME.equals(tableName)) {
         pairs = new MetaTableLocator().getMetaRegionsAndLocations(zookeeper);
       } else {
@@ -1833,10 +1841,10 @@ public class HBaseAdmin implements Admin {
       if (splitPoint == null) {
         LOG.info("SplitPoint is null, will find bestSplitPoint from Region");
       }
-      for (Pair<HRegionInfo, ServerName> pair: pairs) {
+      for (Pair<RegionInfo, ServerName> pair: pairs) {
         // May not be a server for a particular row
         if (pair.getSecond() == null) continue;
-        HRegionInfo r = pair.getFirst();
+        RegionInfo r = pair.getFirst();
         // check for parents
         if (r.isSplitParent()) continue;
         // if a split point given, only split that particular region
@@ -1854,7 +1862,7 @@ public class HBaseAdmin implements Admin {
 
   @Override
   public void splitRegion(final byte[] regionName, final byte [] splitPoint) throws IOException {
-    Pair<HRegionInfo, ServerName> regionServerPair = getRegion(regionName);
+    Pair<RegionInfo, ServerName> regionServerPair = getRegion(regionName);
     if (regionServerPair == null) {
       throw new IllegalArgumentException("Invalid region: " + Bytes.toStringBinary(regionName));
     }
@@ -1921,19 +1929,18 @@ public class HBaseAdmin implements Admin {
    * Throw IllegalArgumentException if <code>regionName</code> is null.
    * @throws IOException
    */
-  Pair<HRegionInfo, ServerName> getRegion(final byte[] regionName) throws IOException {
+  Pair<RegionInfo, ServerName> getRegion(final byte[] regionName) throws IOException {
     if (regionName == null) {
       throw new IllegalArgumentException("Pass a table name or region name");
     }
-    Pair<HRegionInfo, ServerName> pair =
-      MetaTableAccessor.getRegion(connection, regionName);
+    Pair<RegionInfo, ServerName> pair = MetaTableAccessor.getRegion(connection, regionName);
     if (pair == null) {
-      final AtomicReference<Pair<HRegionInfo, ServerName>> result = new AtomicReference<>(null);
+      final AtomicReference<Pair<RegionInfo, ServerName>> result = new AtomicReference<>(null);
       final String encodedName = Bytes.toString(regionName);
       MetaTableAccessor.Visitor visitor = new MetaTableAccessor.Visitor() {
         @Override
         public boolean visit(Result data) throws IOException {
-          HRegionInfo info = MetaTableAccessor.getHRegionInfo(data);
+          RegionInfo info = MetaTableAccessor.getRegionInfo(data);
           if (info == null) {
             LOG.warn("No serialized HRegionInfo in " + data);
             return true;
@@ -1979,7 +1986,7 @@ public class HBaseAdmin implements Admin {
       return HRegionInfo.FIRST_META_REGIONINFO.getRegionName();
     }
     byte[] tmp = regionNameOrEncodedRegionName;
-    Pair<HRegionInfo, ServerName> regionServerPair = getRegion(regionNameOrEncodedRegionName);
+    Pair<RegionInfo, ServerName> regionServerPair = getRegion(regionNameOrEncodedRegionName);
     if (regionServerPair != null && regionServerPair.getFirst() != null) {
       tmp = regionServerPair.getFirst().getRegionName();
     }
@@ -2373,23 +2380,10 @@ public class HBaseAdmin implements Admin {
   @Deprecated
   @Override
   public List<HRegionInfo> getTableRegions(final TableName tableName)
-  throws IOException {
-    ZooKeeperWatcher zookeeper =
-      new ZooKeeperWatcher(conf, ZK_IDENTIFIER_PREFIX + connection.toString(),
-        new ThrowableAbortable());
-    List<HRegionInfo> regions = null;
-    try {
-      if (TableName.META_TABLE_NAME.equals(tableName)) {
-        regions = new MetaTableLocator().getMetaRegions(zookeeper);
-      } else {
-        regions = MetaTableAccessor.getTableRegions(connection, tableName, true);
-      }
-    } finally {
-      zookeeper.close();
-    }
-    return regions == null ? null : regions.stream()
-            .map(hri -> new ImmutableHRegionInfo(hri))
-            .collect(Collectors.toList());
+    throws IOException {
+    return getRegions(tableName).stream()
+        .map(ImmutableHRegionInfo::new)
+        .collect(Collectors.toList());
   }
 
   @Override
@@ -2499,7 +2493,7 @@ public class HBaseAdmin implements Admin {
   @Override
   public CompactionState getCompactionStateForRegion(final byte[] regionName)
   throws IOException {
-    final Pair<HRegionInfo, ServerName> regionServerPair = getRegion(regionName);
+    final Pair<RegionInfo, ServerName> regionServerPair = getRegion(regionName);
     if (regionServerPair == null) {
       throw new IllegalArgumentException("Invalid region: " + Bytes.toStringBinary(regionName));
     }
@@ -3320,7 +3314,7 @@ public class HBaseAdmin implements Admin {
       default:
         ZooKeeperWatcher zookeeper = null;
         try {
-          List<Pair<HRegionInfo, ServerName>> pairs;
+          List<Pair<RegionInfo, ServerName>> pairs;
           if (TableName.META_TABLE_NAME.equals(tableName)) {
             zookeeper = new ZooKeeperWatcher(conf, ZK_IDENTIFIER_PREFIX + connection.toString(),
               new ThrowableAbortable());
@@ -3328,7 +3322,7 @@ public class HBaseAdmin implements Admin {
           } else {
             pairs = MetaTableAccessor.getTableRegionsAndLocations(connection, tableName);
           }
-          for (Pair<HRegionInfo, ServerName> pair: pairs) {
+          for (Pair<RegionInfo, ServerName> pair: pairs) {
             if (pair.getFirst().isOffline()) continue;
             if (pair.getSecond() == null) continue;
             final ServerName sn = pair.getSecond();

http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HRegionLocator.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HRegionLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HRegionLocator.java
index 3416680..71d8b6f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HRegionLocator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HRegionLocator.java
@@ -22,7 +22,6 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
@@ -30,9 +29,11 @@ import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.RegionLocations;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.util.Pair;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.yetus.audience.InterfaceStability;
-import org.apache.hadoop.hbase.util.Pair;
+
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
 
 /**
  * An implementation of {@link RegionLocator}. Used to view region location information for a single
@@ -84,10 +85,10 @@ public class HRegionLocator implements RegionLocator {
   @Override
   public List<HRegionLocation> getAllRegionLocations() throws IOException {
     TableName tableName = getName();
-    List<Pair<HRegionInfo, ServerName>> locations =
+    List<Pair<RegionInfo, ServerName>> locations =
         MetaTableAccessor.getTableRegionsAndLocations(this.connection, tableName);
     ArrayList<HRegionLocation> regions = new ArrayList<>(locations.size());
-    for (Pair<HRegionInfo, ServerName> entry : locations) {
+    for (Pair<RegionInfo, ServerName> entry : locations) {
       regions.add(new HRegionLocation(entry.getFirst(), entry.getSecond()));
 
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java
index 48ae8d6..3a6e3b4 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java
@@ -19,9 +19,6 @@
  */
 package org.apache.hadoop.hbase.client;
 
-import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.hbase.shaded.com.google.common.util.concurrent.ThreadFactoryBuilder;
-
 import java.io.IOException;
 import java.util.AbstractMap.SimpleEntry;
 import java.util.ArrayList;
@@ -43,13 +40,15 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 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.ServerName;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hbase.shaded.com.google.common.util.concurrent.ThreadFactoryBuilder;
 
 /**
  * HTableMultiplexer provides a thread-safe non blocking PUT API across all the tables.
@@ -207,7 +206,7 @@ public class HTableMultiplexer {
         LinkedBlockingQueue<PutStatus> queue = getQueue(loc);
 
         // Generate a MultiPutStatus object and offer it into the queue
-        PutStatus s = new PutStatus(loc.getRegionInfo(), put, maxAttempts);
+        PutStatus s = new PutStatus(loc.getRegion(), put, maxAttempts);
 
         return queue.offer(s);
       }
@@ -372,11 +371,11 @@ public class HTableMultiplexer {
 
   @VisibleForTesting
   static class PutStatus {
-    final HRegionInfo regionInfo;
+    final RegionInfo regionInfo;
     final Put put;
     final int maxAttempCount;
 
-    public PutStatus(HRegionInfo regionInfo, Put put, int maxAttempCount) {
+    public PutStatus(RegionInfo regionInfo, Put put, int maxAttempCount) {
       this.regionInfo = regionInfo;
       this.put = put;
       this.maxAttempCount = maxAttempCount;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ImmutableHRegionInfo.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ImmutableHRegionInfo.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ImmutableHRegionInfo.java
index 16329c8..ed00cee 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ImmutableHRegionInfo.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ImmutableHRegionInfo.java
@@ -33,7 +33,7 @@ public class ImmutableHRegionInfo extends HRegionInfo {
    *
    * @param other
    */
-  public ImmutableHRegionInfo(HRegionInfo other) {
+  public ImmutableHRegionInfo(RegionInfo other) {
     super(other);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
index b1aa940..589796a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
@@ -47,7 +47,6 @@ import org.apache.hadoop.hbase.AsyncMetaTableAccessor;
 import org.apache.hadoop.hbase.ClusterStatus;
 import org.apache.hadoop.hbase.ClusterStatus.Option;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.MetaTableAccessor.QueryType;
@@ -61,7 +60,6 @@ import org.apache.hadoop.hbase.TableNotDisabledException;
 import org.apache.hadoop.hbase.TableNotEnabledException;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.UnknownRegionException;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.client.AsyncRpcRetryingCallerFactory.AdminRequestCallerBuilder;
 import org.apache.hadoop.hbase.client.AsyncRpcRetryingCallerFactory.MasterRequestCallerBuilder;
 import org.apache.hadoop.hbase.client.AsyncRpcRetryingCallerFactory.ServerRequestCallerBuilder;
@@ -78,6 +76,15 @@ import org.apache.hadoop.hbase.quotas.QuotaTableUtil;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
+import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
+import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
+import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.yetus.audience.InterfaceAudience;
+
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback;
 import org.apache.hadoop.hbase.shaded.io.netty.util.Timeout;
 import org.apache.hadoop.hbase.shaded.io.netty.util.TimerTask;
@@ -240,13 +247,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.Remov
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos;
-import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
-import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
-import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
-import org.apache.hadoop.hbase.util.Pair;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.protobuf.Message;
@@ -595,7 +595,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
                   int notDeployed = 0;
                   int regionCount = 0;
                   for (HRegionLocation location : locations) {
-                    HRegionInfo info = location.getRegionInfo();
+                    RegionInfo info = location.getRegionInfo();
                     if (location.getServerName() == null) {
                       if (LOG.isDebugEnabled()) {
                         LOG.debug("Table " + tableName + " has not deployed region "
@@ -739,10 +739,10 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<List<HRegionInfo>> getOnlineRegions(ServerName serverName) {
-    return this.<List<HRegionInfo>> newAdminCaller()
+  public CompletableFuture<List<RegionInfo>> getOnlineRegions(ServerName serverName) {
+    return this.<List<RegionInfo>> newAdminCaller()
         .action((controller, stub) -> this
-            .<GetOnlineRegionRequest, GetOnlineRegionResponse, List<HRegionInfo>> adminCall(
+            .<GetOnlineRegionRequest, GetOnlineRegionResponse, List<RegionInfo>> adminCall(
               controller, stub, RequestConverter.buildGetOnlineRegionRequest(),
               (s, c, req, done) -> s.getOnlineRegion(c, req, done),
               resp -> ProtobufUtil.getRegionInfos(resp)))
@@ -750,7 +750,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<List<HRegionInfo>> getTableRegions(TableName tableName) {
+  public CompletableFuture<List<RegionInfo>> getTableRegions(TableName tableName) {
     if (tableName.equals(META_TABLE_NAME)) {
       return connection.getLocator().getRegionLocation(tableName, null, null, operationTimeoutNs)
           .thenApply(loc -> Arrays.asList(loc.getRegionInfo()));
@@ -807,7 +807,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
           return;
         }
 
-        HRegionInfo regionInfo = location.getRegionInfo();
+        RegionInfo regionInfo = location.getRegionInfo();
         this.<Void> newAdminCaller()
             .serverName(serverName)
             .action(
@@ -973,7 +973,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
   /**
    * Compact the region at specific region server.
    */
-  private CompletableFuture<Void> compact(final ServerName sn, final HRegionInfo hri,
+  private CompletableFuture<Void> compact(final ServerName sn, final RegionInfo hri,
       final boolean major, Optional<byte[]> columnFamily) {
     return this
         .<Void> newAdminCaller()
@@ -987,8 +987,8 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
 
   private byte[] toEncodeRegionName(byte[] regionName) {
     try {
-      return HRegionInfo.isEncodedRegionName(regionName) ? regionName
-          : Bytes.toBytes(HRegionInfo.encodeRegionName(regionName));
+      return RegionInfo.isEncodedRegionName(regionName) ? regionName
+          : Bytes.toBytes(RegionInfo.encodeRegionName(regionName));
     } catch (IOException e) {
       return regionName;
     }
@@ -1002,8 +1002,8 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
           result.completeExceptionally(err);
           return;
         }
-        HRegionInfo regionInfo = location.getRegionInfo();
-        if (regionInfo.getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) {
+        RegionInfo regionInfo = location.getRegionInfo();
+        if (regionInfo.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) {
           result.completeExceptionally(new IllegalArgumentException(
               "Can't invoke merge on non-default regions directly"));
           return;
@@ -1138,14 +1138,14 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
             if (results != null && !results.isEmpty()) {
               List<CompletableFuture<Void>> splitFutures = new ArrayList<>();
               for (Result r : results) {
-                if (r.isEmpty() || MetaTableAccessor.getHRegionInfo(r) == null) continue;
+                if (r.isEmpty() || MetaTableAccessor.getRegionInfo(r) == null) continue;
                 RegionLocations rl = MetaTableAccessor.getRegionLocations(r);
                 if (rl != null) {
                   for (HRegionLocation h : rl.getRegionLocations()) {
                     if (h != null && h.getServerName() != null) {
-                      HRegionInfo hri = h.getRegionInfo();
+                      RegionInfo hri = h.getRegion();
                       if (hri == null || hri.isSplitParent()
-                          || hri.getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID)
+                          || hri.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID)
                         continue;
                       splitFutures.add(split(hri, Optional.empty()));
                     }
@@ -1202,8 +1202,8 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
     CompletableFuture<Void> future = new CompletableFuture<>();
     getRegionLocation(regionName).whenComplete(
       (location, err) -> {
-        HRegionInfo regionInfo = location.getRegionInfo();
-        if (regionInfo.getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) {
+        RegionInfo regionInfo = location.getRegionInfo();
+        if (regionInfo.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) {
           future.completeExceptionally(new IllegalArgumentException(
               "Can't split replicas directly. "
                   + "Replicas are auto-split when their primary is split."));
@@ -1226,7 +1226,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
     return future;
   }
 
-  private CompletableFuture<Void> split(final HRegionInfo hri,
+  private CompletableFuture<Void> split(final RegionInfo hri,
       Optional<byte[]> splitPoint) {
     if (hri.getStartKey() != null && splitPoint.isPresent()
         && Bytes.compareTo(hri.getStartKey(), splitPoint.get()) == 0) {
@@ -2051,7 +2051,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
     }
     try {
       CompletableFuture<Optional<HRegionLocation>> future;
-      if (HRegionInfo.isEncodedRegionName(regionNameOrEncodedRegionName)) {
+      if (RegionInfo.isEncodedRegionName(regionNameOrEncodedRegionName)) {
         future = AsyncMetaTableAccessor.getRegionLocationWithEncodedName(metaTable,
           regionNameOrEncodedRegionName);
       } else {
@@ -2087,19 +2087,19 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
    * @param regionNameOrEncodedRegionName
    * @return region info, wrapped by a {@link CompletableFuture}
    */
-  private CompletableFuture<HRegionInfo> getRegionInfo(byte[] regionNameOrEncodedRegionName) {
+  private CompletableFuture<RegionInfo> getRegionInfo(byte[] regionNameOrEncodedRegionName) {
     if (regionNameOrEncodedRegionName == null) {
       return failedFuture(new IllegalArgumentException("Passed region name can't be null"));
     }
 
     if (Bytes.equals(regionNameOrEncodedRegionName,
-      HRegionInfo.FIRST_META_REGIONINFO.getRegionName())
+      RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName())
         || Bytes.equals(regionNameOrEncodedRegionName,
-          HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes())) {
-      return CompletableFuture.completedFuture(HRegionInfo.FIRST_META_REGIONINFO);
+        RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedNameAsBytes())) {
+      return CompletableFuture.completedFuture(RegionInfoBuilder.FIRST_META_REGIONINFO);
     }
 
-    CompletableFuture<HRegionInfo> future = new CompletableFuture<>();
+    CompletableFuture<RegionInfo> future = new CompletableFuture<>();
     getRegionLocation(regionNameOrEncodedRegionName).whenComplete((location, err) -> {
       if (err != null) {
         future.completeExceptionally(err);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTable.java
index 4b3d44b..cd0226b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTable.java
@@ -17,16 +17,15 @@
  */
 package org.apache.hadoop.hbase.client;
 
-import com.google.protobuf.RpcCallback;
-import com.google.protobuf.RpcChannel;
-import com.google.protobuf.RpcController;
-
 import java.util.concurrent.CompletableFuture;
 import java.util.function.Function;
 
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.yetus.audience.InterfaceAudience;
 
+import com.google.protobuf.RpcCallback;
+import com.google.protobuf.RpcChannel;
+import com.google.protobuf.RpcController;
+
 /**
  * A low level asynchronous table.
  * <p>
@@ -134,10 +133,10 @@ public interface RawAsyncTable extends AsyncTableBase {
    * As the locating itself also takes some time, the implementation may want to send rpc calls on
    * the fly, which means we do not know how many regions we have when we get the return value of
    * the rpc calls, so we need an {@link #onComplete()} which is used to tell you that we have
-   * passed all the return values to you(through the {@link #onRegionComplete(HRegionInfo, Object)}
-   * or {@link #onRegionError(HRegionInfo, Throwable)} calls), i.e, there will be no
-   * {@link #onRegionComplete(HRegionInfo, Object)} or
-   * {@link #onRegionError(HRegionInfo, Throwable)} calls in the future.
+   * passed all the return values to you(through the {@link #onRegionComplete(RegionInfo, Object)}
+   * or {@link #onRegionError(RegionInfo, Throwable)} calls), i.e, there will be no
+   * {@link #onRegionComplete(RegionInfo, Object)} or
+   * {@link #onRegionError(RegionInfo, Throwable)} calls in the future.
    * <p>
    * Here is a pseudo code to describe a typical implementation of a range coprocessor service
    * method to help you better understand how the {@link CoprocessorCallback} will be called. The
@@ -179,18 +178,18 @@ public interface RawAsyncTable extends AsyncTableBase {
      * @param region the region that the response belongs to
      * @param resp the response of the coprocessor call
      */
-    void onRegionComplete(HRegionInfo region, R resp);
+    void onRegionComplete(RegionInfo region, R resp);
 
     /**
      * @param region the region that the error belongs to
      * @param error the response error of the coprocessor call
      */
-    void onRegionError(HRegionInfo region, Throwable error);
+    void onRegionError(RegionInfo region, Throwable error);
 
     /**
      * Indicate that all responses of the regions have been notified by calling
-     * {@link #onRegionComplete(HRegionInfo, Object)} or
-     * {@link #onRegionError(HRegionInfo, Throwable)}.
+     * {@link #onRegionComplete(RegionInfo, Object)} or
+     * {@link #onRegionError(RegionInfo, Throwable)}.
      */
     void onComplete();
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTableImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTableImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTableImpl.java
index 735ff30..722ee26 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTableImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTableImpl.java
@@ -23,8 +23,6 @@ import static org.apache.hadoop.hbase.HConstants.EMPTY_START_ROW;
 import static org.apache.hadoop.hbase.client.ConnectionUtils.checkHasFamilies;
 import static org.apache.hadoop.hbase.client.ConnectionUtils.isEmptyStopRow;
 
-import com.google.protobuf.RpcChannel;
-
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -38,13 +36,15 @@ import java.util.function.Function;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.CompareOperator;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.client.AsyncRpcRetryingCallerFactory.SingleRequestCallerBuilder;
 import org.apache.hadoop.hbase.filter.BinaryComparator;
 import org.apache.hadoop.hbase.ipc.HBaseRpcController;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.ReflectionUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
@@ -58,8 +58,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateRequ
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionAction;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.CompareType;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.ReflectionUtils;
+
+import com.google.protobuf.RpcChannel;
 
 /**
  * The implementation of RawAsyncTable.
@@ -439,7 +439,7 @@ class RawAsyncTableImpl implements RawAsyncTable {
   }
 
   private <S, R> CompletableFuture<R> coprocessorService(Function<RpcChannel, S> stubMaker,
-      CoprocessorCallable<S, R> callable, HRegionInfo region, byte[] row) {
+      CoprocessorCallable<S, R> callable, RegionInfo region, byte[] row) {
     RegionCoprocessorRpcChannelImpl channel = new RegionCoprocessorRpcChannelImpl(conn, tableName,
         region, row, rpcTimeoutNs, operationTimeoutNs);
     S stub = stubMaker.apply(channel);
@@ -461,7 +461,7 @@ class RawAsyncTableImpl implements RawAsyncTable {
     return coprocessorService(stubMaker, callable, null, row);
   }
 
-  private boolean locateFinished(HRegionInfo region, byte[] endKey, boolean endKeyInclusive) {
+  private boolean locateFinished(RegionInfo region, byte[] endKey, boolean endKeyInclusive) {
     if (isEmptyStopRow(endKey)) {
       if (isEmptyStopRow(region.getEndKey())) {
         return true;
@@ -488,7 +488,7 @@ class RawAsyncTableImpl implements RawAsyncTable {
       return;
     }
     unfinishedRequest.incrementAndGet();
-    HRegionInfo region = loc.getRegionInfo();
+    RegionInfo region = loc.getRegionInfo();
     if (locateFinished(region, endKey, endKeyInclusive)) {
       locateFinished.set(true);
     } else {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionCoprocessorRpcChannelImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionCoprocessorRpcChannelImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionCoprocessorRpcChannelImpl.java
index ee30c16..d7684c2 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionCoprocessorRpcChannelImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionCoprocessorRpcChannelImpl.java
@@ -17,27 +17,27 @@
  */
 package org.apache.hadoop.hbase.client;
 
-import com.google.protobuf.Descriptors.MethodDescriptor;
-import com.google.protobuf.Message;
-import com.google.protobuf.RpcCallback;
-import com.google.protobuf.RpcChannel;
-import com.google.protobuf.RpcController;
-
 import java.io.IOException;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.hadoop.hbase.DoNotRetryIOException;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
 import org.apache.hadoop.hbase.ipc.HBaseRpcController;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse;
-import org.apache.hadoop.hbase.util.Bytes;
+
+import com.google.protobuf.Descriptors.MethodDescriptor;
+import com.google.protobuf.Message;
+import com.google.protobuf.RpcCallback;
+import com.google.protobuf.RpcChannel;
+import com.google.protobuf.RpcController;
 
 /**
  * The implementation of a region based coprocessor rpc channel.
@@ -49,7 +49,7 @@ class RegionCoprocessorRpcChannelImpl implements RpcChannel {
 
   private final TableName tableName;
 
-  private final HRegionInfo region;
+  private final RegionInfo region;
 
   private final byte[] row;
 
@@ -57,7 +57,7 @@ class RegionCoprocessorRpcChannelImpl implements RpcChannel {
 
   private final long operationTimeoutNs;
 
-  RegionCoprocessorRpcChannelImpl(AsyncConnectionImpl conn, TableName tableName, HRegionInfo region,
+  RegionCoprocessorRpcChannelImpl(AsyncConnectionImpl conn, TableName tableName, RegionInfo region,
       byte[] row, long rpcTimeoutNs, long operationTimeoutNs) {
     this.conn = conn;
     this.tableName = tableName;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java
index 3646722..e9716e9 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java
@@ -18,17 +18,6 @@
  */
 package org.apache.hadoop.hbase.client;
 
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.MD5Hash;
-import org.apache.hadoop.io.DataInputBuffer;
-import org.apache.hadoop.util.StringUtils;
-import org.apache.yetus.audience.InterfaceAudience;
-
 import java.io.DataInputStream;
 import java.io.IOException;
 import java.util.ArrayList;
@@ -36,9 +25,21 @@ import java.util.Arrays;
 import java.util.Comparator;
 import java.util.List;
 import java.util.stream.Collectors;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.util.ByteArrayHashKey;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.HashKey;
 import org.apache.hadoop.hbase.util.JenkinsHash;
+import org.apache.hadoop.hbase.util.MD5Hash;
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
 
 /**
  * Information about a region. A region is a range of keys in the whole keyspace
@@ -436,7 +437,7 @@ public interface RegionInfo {
    * @see #parseFrom(byte[])
    */
   static byte [] toByteArray(RegionInfo ri) {
-    byte [] bytes = ProtobufUtil.toProtoRegionInfo(ri).toByteArray();
+    byte [] bytes = ProtobufUtil.toRegionInfo(ri).toByteArray();
     return ProtobufUtil.prependPBMagic(bytes);
   }
 
@@ -691,7 +692,7 @@ public interface RegionInfo {
    * @throws IOException
    */
   static byte [] toDelimitedByteArray(RegionInfo ri) throws IOException {
-    return ProtobufUtil.toDelimitedByteArray(ProtobufUtil.toProtoRegionInfo(ri));
+    return ProtobufUtil.toDelimitedByteArray(ProtobufUtil.toRegionInfo(ri));
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java
index a76767d..7d5c476 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java
@@ -18,16 +18,17 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import java.util.Arrays;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.util.ArrayUtils;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.yetus.audience.InterfaceAudience;
 
-import java.util.Arrays;
-
 @InterfaceAudience.Private
 public class RegionInfoBuilder {
   private static final Log LOG = LogFactory.getLog(RegionInfoBuilder.class);
@@ -52,7 +53,15 @@ public class RegionInfoBuilder {
   public static final RegionInfo FIRST_META_REGIONINFO =
     new MutableRegionInfo(1L, TableName.META_TABLE_NAME, RegionInfo.DEFAULT_REPLICA_ID);
 
-  private MutableRegionInfo content = null;
+  private final TableName tableName;
+  private byte[] startKey = HConstants.EMPTY_START_ROW;
+  private byte[] endKey = HConstants.EMPTY_END_ROW;
+  private long regionId = System.currentTimeMillis();
+  private int replicaId = RegionInfo.DEFAULT_REPLICA_ID;
+  private boolean offLine = false;
+  private boolean split = false;
+  private byte[] regionName = null;
+  private String encodedName = null;
 
   public static RegionInfoBuilder newBuilder(TableName tableName) {
     return new RegionInfoBuilder(tableName);
@@ -63,52 +72,54 @@ public class RegionInfoBuilder {
   }
 
   private RegionInfoBuilder(TableName tableName) {
-    this.content = new MutableRegionInfo(tableName);
+    this.tableName = tableName;
   }
 
   private RegionInfoBuilder(RegionInfo regionInfo) {
-    this.content = new MutableRegionInfo(regionInfo);
+    this.tableName = regionInfo.getTable();
+    this.startKey = regionInfo.getStartKey();
+    this.endKey = regionInfo.getEndKey();
+    this.offLine = regionInfo.isOffline();
+    this.split = regionInfo.isSplit();
+    this.regionId = regionInfo.getRegionId();
+    this.replicaId = regionInfo.getReplicaId();
+    this.regionName = regionInfo.getRegionName();
+    this.encodedName = regionInfo.getEncodedName();
   }
 
   public RegionInfoBuilder setStartKey(byte[] startKey) {
-    content.setStartKey(startKey);
+    this.startKey = startKey;
     return this;
   }
 
   public RegionInfoBuilder setEndKey(byte[] endKey) {
-    content.setEndKey(endKey);
+    this.endKey = endKey;
     return this;
   }
 
   public RegionInfoBuilder setRegionId(long regionId) {
-    content.setRegionId(regionId);
+    this.regionId = regionId;
     return this;
   }
 
   public RegionInfoBuilder setReplicaId(int replicaId) {
-    content.setReplicaId(replicaId);
+    this.replicaId = replicaId;
     return this;
   }
 
-  public RegionInfoBuilder setSplit(boolean isSplit) {
-    content.setSplit(isSplit);
+  public RegionInfoBuilder setSplit(boolean split) {
+    this.split = split;
     return this;
   }
 
-  public RegionInfoBuilder setOffline(boolean isOffline) {
-    content.setOffline(isOffline);
+  public RegionInfoBuilder setOffline(boolean offLine) {
+    this.offLine = offLine;
     return this;
   }
 
   public RegionInfo build() {
-    RegionInfo ri = new MutableRegionInfo(content);
-    // Run a late check that we are not creating default meta region.
-    if (ri.getTable().equals(TableName.META_TABLE_NAME) &&
-        ri.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) {
-      throw new IllegalArgumentException("Cannot create the default meta region; " +
-        "use static define FIRST_META_REGIONINFO");
-    }
-    return new MutableRegionInfo(content);
+    return new MutableRegionInfo(tableName, startKey, endKey, split,
+        regionId, replicaId, offLine, regionName, encodedName);
   }
 
   /**
@@ -144,26 +155,49 @@ public class RegionInfoBuilder {
     // but now table state is kept up in zookeeper as of 0.90.0 HBase.
     private boolean offLine = false;
     private boolean split = false;
-    private long regionId = -1;
-    private int replicaId = RegionInfo.DEFAULT_REPLICA_ID;
-    private transient byte [] regionName = HConstants.EMPTY_BYTE_ARRAY;
-    private byte [] startKey = HConstants.EMPTY_BYTE_ARRAY;
-    private byte [] endKey = HConstants.EMPTY_BYTE_ARRAY;
-    private int hashCode = -1;
-    private String encodedName;
-    private byte [] encodedNameAsBytes;
-    // Current TableName
-    private TableName tableName;
-
-    private void setHashCode() {
-      int result = Arrays.hashCode(this.regionName);
-      result ^= this.regionId;
-      result ^= Arrays.hashCode(this.startKey);
-      result ^= Arrays.hashCode(this.endKey);
-      result ^= Boolean.valueOf(this.offLine).hashCode();
-      result ^= Arrays.hashCode(this.tableName.getName());
-      result ^= this.replicaId;
-      this.hashCode = result;
+    private final long regionId;
+    private final int replicaId;
+    private final byte[] regionName;
+    private final byte[] startKey;
+    private final byte[] endKey;
+    private final int hashCode;
+    private final String encodedName;
+    private final byte[] encodedNameAsBytes;
+    private final TableName tableName;
+
+    private static int generateHashCode(final TableName tableName, final byte[] startKey,
+        final byte[] endKey, final long regionId,
+        final int replicaId, boolean offLine, byte[] regionName) {
+      int result = Arrays.hashCode(regionName);
+      result ^= regionId;
+      result ^= Arrays.hashCode(checkStartKey(startKey));
+      result ^= Arrays.hashCode(checkEndKey(endKey));
+      result ^= Boolean.valueOf(offLine).hashCode();
+      result ^= Arrays.hashCode(tableName.getName());
+      result ^= replicaId;
+      return result;
+    }
+
+    private static byte[] checkStartKey(byte[] startKey) {
+      return startKey == null? HConstants.EMPTY_START_ROW: startKey;
+    }
+
+    private static byte[] checkEndKey(byte[] endKey) {
+      return endKey == null? HConstants.EMPTY_END_ROW: endKey;
+    }
+
+    private static TableName checkTableName(TableName tableName) {
+      if (tableName == null) {
+        throw new IllegalArgumentException("TableName cannot be null");
+      }
+      return tableName;
+    }
+
+    private static int checkReplicaId(int regionId) {
+      if (regionId > MAX_REPLICA_ID) {
+        throw new IllegalArgumentException("ReplicaId cannot be greater than" + MAX_REPLICA_ID);
+      }
+      return regionId;
     }
 
     /**
@@ -171,162 +205,57 @@ public class RegionInfoBuilder {
      * first meta regions
      */
     private MutableRegionInfo(long regionId, TableName tableName, int replicaId) {
-      // This constructor is currently private for making hbase:meta region only.
-      super();
-      this.regionId = regionId;
-      this.tableName = tableName;
-      this.replicaId = replicaId;
-      // Note: First Meta region replicas names are in old format so we pass false here.
-      this.regionName =
-        RegionInfo.createRegionName(tableName, null, regionId, replicaId, false);
-      setHashCode();
-    }
-
-    MutableRegionInfo(final TableName tableName) {
-      this(tableName, null, null);
+      this(tableName,
+          HConstants.EMPTY_START_ROW,
+          HConstants.EMPTY_END_ROW,
+          false,
+          regionId,
+          replicaId,
+          false,
+          RegionInfo.createRegionName(tableName, null, regionId, replicaId, false));
     }
 
-    /**
-     * Construct MutableRegionInfo with explicit parameters
-     *
-     * @param tableName the table name
-     * @param startKey first key in region
-     * @param endKey end of key range
-     * @throws IllegalArgumentException
-     */
-    MutableRegionInfo(final TableName tableName, final byte[] startKey, final byte[] endKey)
-    throws IllegalArgumentException {
-      this(tableName, startKey, endKey, false);
-    }
-
-    /**
-     * Construct MutableRegionInfo with explicit parameters
-     *
-     * @param tableName the table descriptor
-     * @param startKey first key in region
-     * @param endKey end of key range
-     * @param split true if this region has split and we have daughter regions
-     * regions that may or may not hold references to this region.
-     * @throws IllegalArgumentException
-     */
-    MutableRegionInfo(final TableName tableName, final byte[] startKey, final byte[] endKey,
-        final boolean split)
-    throws IllegalArgumentException {
-      this(tableName, startKey, endKey, split, System.currentTimeMillis());
-    }
-
-    /**
-     * Construct MutableRegionInfo with explicit parameters
-     *
-     * @param tableName the table descriptor
-     * @param startKey first key in region
-     * @param endKey end of key range
-     * @param split true if this region has split and we have daughter regions
-     * regions that may or may not hold references to this region.
-     * @param regionid Region id to use.
-     * @throws IllegalArgumentException
-     */
     MutableRegionInfo(final TableName tableName, final byte[] startKey,
-                       final byte[] endKey, final boolean split, final long regionid)
-    throws IllegalArgumentException {
-      this(tableName, startKey, endKey, split, regionid, RegionInfo.DEFAULT_REPLICA_ID);
+        final byte[] endKey, final boolean split, final long regionId,
+        final int replicaId, boolean offLine, byte[] regionName) {
+      this(checkTableName(tableName),
+          checkStartKey(startKey),
+          checkEndKey(endKey),
+          split, regionId,
+          checkReplicaId(replicaId),
+          offLine,
+          regionName,
+          RegionInfo.encodeRegionName(regionName));
     }
 
-    /**
-     * Construct MutableRegionInfo with explicit parameters
-     *
-     * @param tableName the table descriptor
-     * @param startKey first key in region
-     * @param endKey end of key range
-     * @param split true if this region has split and we have daughter regions
-     * regions that may or may not hold references to this region.
-     * @param regionid Region id to use.
-     * @param replicaId the replicaId to use
-     * @throws IllegalArgumentException
-     */
     MutableRegionInfo(final TableName tableName, final byte[] startKey,
-                       final byte[] endKey, final boolean split, final long regionid,
-                       final int replicaId)
-      throws IllegalArgumentException {
-      super();
-      if (tableName == null) {
-        throw new IllegalArgumentException("TableName cannot be null");
-      }
-      this.tableName = tableName;
-      this.offLine = false;
-      this.regionId = regionid;
-      this.replicaId = replicaId;
-      if (this.replicaId > MAX_REPLICA_ID) {
-        throw new IllegalArgumentException("ReplicaId cannot be greater than" + MAX_REPLICA_ID);
-      }
-
-      this.regionName = RegionInfo.createRegionName(this.tableName, startKey, regionId, replicaId,
-        !this.tableName.equals(TableName.META_TABLE_NAME));
-
+        final byte[] endKey, final boolean split, final long regionId,
+        final int replicaId, boolean offLine, byte[] regionName, String encodedName) {
+      this.tableName = checkTableName(tableName);
+      this.startKey = checkStartKey(startKey);
+      this.endKey = checkEndKey(endKey);
       this.split = split;
-      this.endKey = endKey == null? HConstants.EMPTY_END_ROW: endKey.clone();
-      this.startKey = startKey == null?
-        HConstants.EMPTY_START_ROW: startKey.clone();
-      this.tableName = tableName;
-      setHashCode();
-    }
-
-    /**
-     * Construct MutableRegionInfo.
-     * Only for RegionInfoBuilder to use.
-     * @param other
-     */
-    MutableRegionInfo(MutableRegionInfo other, boolean isMetaRegion) {
-      super();
-      if (other.getTable() == null) {
-        throw new IllegalArgumentException("TableName cannot be null");
-      }
-      this.tableName = other.getTable();
-      this.offLine = other.isOffline();
-      this.regionId = other.getRegionId();
-      this.replicaId = other.getReplicaId();
-      if (this.replicaId > MAX_REPLICA_ID) {
-        throw new IllegalArgumentException("ReplicaId cannot be greater than" + MAX_REPLICA_ID);
-      }
-
-      if(isMetaRegion) {
-        // Note: First Meta region replicas names are in old format
-        this.regionName = RegionInfo.createRegionName(
-                other.getTable(), null, other.getRegionId(),
-                other.getReplicaId(), false);
+      this.regionId = regionId;
+      this.replicaId = checkReplicaId(replicaId);
+      this.offLine = offLine;
+      if (ArrayUtils.isEmpty(regionName)) {
+        this.regionName = RegionInfo.createRegionName(this.tableName, this.startKey, this.regionId, this.replicaId,
+            !this.tableName.equals(TableName.META_TABLE_NAME));
+        this.encodedName = RegionInfo.encodeRegionName(this.regionName);
       } else {
-        this.regionName = RegionInfo.createRegionName(
-                other.getTable(), other.getStartKey(), other.getRegionId(),
-                other.getReplicaId(), true);
+        this.regionName = regionName;
+        this.encodedName = encodedName;
       }
-
-      this.split = other.isSplit();
-      this.endKey = other.getEndKey() == null? HConstants.EMPTY_END_ROW: other.getEndKey().clone();
-      this.startKey = other.getStartKey() == null?
-        HConstants.EMPTY_START_ROW: other.getStartKey().clone();
-      this.tableName = other.getTable();
-      setHashCode();
-    }
-
-    /**
-     * Construct a copy of RegionInfo as MutableRegionInfo.
-     * Only for RegionInfoBuilder to use.
-     * @param regionInfo
-     */
-    MutableRegionInfo(RegionInfo regionInfo) {
-      super();
-      this.endKey = regionInfo.getEndKey();
-      this.offLine = regionInfo.isOffline();
-      this.regionId = regionInfo.getRegionId();
-      this.regionName = regionInfo.getRegionName();
-      this.split = regionInfo.isSplit();
-      this.startKey = regionInfo.getStartKey();
-      this.hashCode = regionInfo.hashCode();
-      this.encodedName = regionInfo.getEncodedName();
-      this.tableName = regionInfo.getTable();
-      this.replicaId = regionInfo.getReplicaId();
+      this.hashCode = generateHashCode(
+          this.tableName,
+          this.startKey,
+          this.endKey,
+          this.regionId,
+          this.replicaId,
+          this.offLine,
+          this.regionName);
+      this.encodedNameAsBytes = Bytes.toBytes(this.encodedName);
     }
-
     /**
      * @return Return a short, printable name for this region
      * (usually encoded name) for us logging.
@@ -342,15 +271,6 @@ public class RegionInfoBuilder {
       return regionId;
     }
 
-    /**
-     * set region id.
-     * @param regionId
-     * @return MutableRegionInfo
-     */
-    public MutableRegionInfo setRegionId(long regionId) {
-      this.regionId = regionId;
-      return this;
-    }
 
     /**
      * @return the regionName as an array of bytes.
@@ -362,16 +282,6 @@ public class RegionInfoBuilder {
     }
 
     /**
-     * set region name.
-     * @param regionName
-     * @return MutableRegionInfo
-     */
-    public MutableRegionInfo setRegionName(byte[] regionName) {
-      this.regionName = regionName;
-      return this;
-    }
-
-    /**
      * @return Region name as a String for use in logging, etc.
      */
     @Override
@@ -389,18 +299,12 @@ public class RegionInfoBuilder {
 
     /** @return the encoded region name */
     @Override
-    public synchronized String getEncodedName() {
-      if (this.encodedName == null) {
-        this.encodedName = RegionInfo.encodeRegionName(this.regionName);
-      }
+    public String getEncodedName() {
       return this.encodedName;
     }
 
     @Override
-    public synchronized byte [] getEncodedNameAsBytes() {
-      if (this.encodedNameAsBytes == null) {
-        this.encodedNameAsBytes = Bytes.toBytes(getEncodedName());
-      }
+    public byte [] getEncodedNameAsBytes() {
       return this.encodedNameAsBytes;
     }
 
@@ -410,14 +314,6 @@ public class RegionInfoBuilder {
       return startKey;
     }
 
-    /**
-     * @param startKey
-     * @return MutableRegionInfo
-     */
-    public MutableRegionInfo setStartKey(byte[] startKey) {
-      this.startKey = startKey;
-      return this;
-    }
 
     /** @return the endKey */
     @Override
@@ -426,26 +322,11 @@ public class RegionInfoBuilder {
     }
 
     /**
-     * @param endKey
-     * @return MutableRegionInfo
-     */
-    public MutableRegionInfo setEndKey(byte[] endKey) {
-      this.endKey = endKey;
-      return this;
-    }
-
-    /**
      * Get current table name of the region
      * @return TableName
      */
     @Override
     public TableName getTable() {
-      // This method name should be getTableName but there was already a method getTableName
-      // that returned a byte array.  It is unfortunate given everywhere else, getTableName returns
-      // a TableName instance.
-      if (tableName == null || tableName.getName().length == 0) {
-        tableName = RegionInfo.getTable(getRegionName());
-      }
       return this.tableName;
     }
 
@@ -560,11 +441,6 @@ public class RegionInfoBuilder {
       return replicaId;
     }
 
-    public MutableRegionInfo setReplicaId(int replicaId) {
-      this.replicaId = replicaId;
-      return this;
-    }
-
     /**
      * @see java.lang.Object#toString()
      */

http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionReplicaUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionReplicaUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionReplicaUtil.java
index 2c1d478..d444c82 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionReplicaUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionReplicaUtil.java
@@ -21,9 +21,8 @@ package org.apache.hadoop.hbase.client;
 import java.util.Collection;
 import java.util.Iterator;
 
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Utility methods which contain the logic for regions and replicas.
@@ -50,30 +49,6 @@ public class RegionReplicaUtil {
   static final int DEFAULT_REPLICA_ID = 0;
 
   /**
-   * Returns the HRegionInfo for the given replicaId. HRegionInfo's correspond to
-   * a range of a table, but more than one "instance" of the same range can be
-   * deployed which are differentiated by the replicaId.
-   * @param replicaId the replicaId to use
-   * @return an HRegionInfo object corresponding to the same range (table, start and
-   * end key), but for the given replicaId.
-   */
-  @Deprecated // Deprecated for HBase-2.0.0, use #getRegionInfoForReplica
-  public static HRegionInfo getRegionInfoForReplica(HRegionInfo regionInfo, int replicaId) {
-    if (regionInfo.getReplicaId() == replicaId) {
-      return regionInfo;
-    }
-    HRegionInfo replicaInfo;
-    if (regionInfo.isMetaRegion()) {
-      replicaInfo = new HRegionInfo(regionInfo.getRegionId(), regionInfo.getTable(), replicaId);
-    } else {
-      replicaInfo = new HRegionInfo(regionInfo.getTable(), regionInfo.getStartKey(),
-        regionInfo.getEndKey(), regionInfo.isSplit(), regionInfo.getRegionId(), replicaId);
-    }
-    replicaInfo.setOffline(regionInfo.isOffline());
-    return replicaInfo;
-  }
-
-  /**
    * Returns the RegionInfo for the given replicaId.
    * RegionInfo's correspond to a range of a table, but more than one
    * "instance" of the same range can be deployed which are differentiated by
@@ -87,31 +62,33 @@ public class RegionReplicaUtil {
     if (regionInfo.getReplicaId() == replicaId) {
       return regionInfo;
     }
-    RegionInfoBuilder replicaInfo;
-    RegionInfo ri;
+
     if (regionInfo.isMetaRegion()) {
-      ri = RegionInfoBuilder.FIRST_META_REGIONINFO;
+      return RegionInfoBuilder.newBuilder(regionInfo.getTable())
+          .setRegionId(regionInfo.getRegionId())
+          .setReplicaId(replicaId)
+          .setOffline(regionInfo.isOffline())
+          .build();
     } else {
-      replicaInfo = RegionInfoBuilder.newBuilder(regionInfo.getTable())
+      return RegionInfoBuilder.newBuilder(regionInfo.getTable())
               .setStartKey(regionInfo.getStartKey())
               .setEndKey(regionInfo.getEndKey())
               .setSplit(regionInfo.isSplit())
               .setRegionId(regionInfo.getRegionId())
-              .setReplicaId(replicaId);
-      replicaInfo.setOffline(regionInfo.isOffline());
-      ri = replicaInfo.build();
+              .setReplicaId(replicaId)
+              .setOffline(regionInfo.isOffline())
+              .build();
     }
-    return ri;
   }
 
   /**
-   * Returns the HRegionInfo for the default replicaId (0). HRegionInfo's correspond to
+   * Returns the RegionInfo for the default replicaId (0). RegionInfo's correspond to
    * a range of a table, but more than one "instance" of the same range can be
    * deployed which are differentiated by the replicaId.
-   * @return an HRegionInfo object corresponding to the same range (table, start and
+   * @return an RegionInfo object corresponding to the same range (table, start and
    * end key), but for the default replicaId.
    */
-  public static HRegionInfo getRegionInfoForDefaultReplica(HRegionInfo regionInfo) {
+  public static RegionInfo getRegionInfoForDefaultReplica(RegionInfo regionInfo) {
     return getRegionInfoForReplica(regionInfo, DEFAULT_REPLICA_ID);
   }
 
@@ -121,7 +98,7 @@ public class RegionReplicaUtil {
   }
 
   /** @return true if this region is a default replica for the region */
-  public static boolean isDefaultReplica(HRegionInfo hri) {
+  public static boolean isDefaultReplica(RegionInfo hri) {
     return  hri.getReplicaId() == DEFAULT_REPLICA_ID;
   }
 
@@ -129,22 +106,22 @@ public class RegionReplicaUtil {
    * Removes the non-default replicas from the passed regions collection
    * @param regions
    */
-  public static void removeNonDefaultRegions(Collection<HRegionInfo> regions) {
-    Iterator<HRegionInfo> iterator = regions.iterator();
+  public static void removeNonDefaultRegions(Collection<RegionInfo> regions) {
+    Iterator<RegionInfo> iterator = regions.iterator();
     while (iterator.hasNext()) {
-      HRegionInfo hri = iterator.next();
+      RegionInfo hri = iterator.next();
       if (!RegionReplicaUtil.isDefaultReplica(hri)) {
         iterator.remove();
       }
     }
   }
 
-  public static boolean isReplicasForSameRegion(HRegionInfo regionInfoA, HRegionInfo regionInfoB) {
+  public static boolean isReplicasForSameRegion(RegionInfo regionInfoA, RegionInfo regionInfoB) {
     return compareRegionInfosWithoutReplicaId(regionInfoA, regionInfoB) == 0;
   }
 
-  private static int compareRegionInfosWithoutReplicaId(HRegionInfo regionInfoA,
-      HRegionInfo regionInfoB) {
+  private static int compareRegionInfosWithoutReplicaId(RegionInfo regionInfoA,
+      RegionInfo regionInfoB) {
     int result = regionInfoA.getTable().compareTo(regionInfoB.getTable());
     if (result != 0) {
       return result;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperRegistry.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperRegistry.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperRegistry.java
index bbd726b..746382f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperRegistry.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperRegistry.java
@@ -22,14 +22,13 @@ import java.util.List;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.RegionLocations;
 import org.apache.hadoop.hbase.ServerName;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.zookeeper.KeeperException;
 
 /**
@@ -76,8 +75,8 @@ class ZooKeeperRegistry implements Registry {
       HRegionLocation[] locs = new HRegionLocation[servers.size()];
       int i = 0;
       for (ServerName server : servers) {
-        HRegionInfo h = RegionReplicaUtil.getRegionInfoForReplica(
-                HRegionInfo.FIRST_META_REGIONINFO, i);
+        RegionInfo h = RegionReplicaUtil.getRegionInfoForReplica(
+                RegionInfoBuilder.FIRST_META_REGIONINFO, i);
         if (server == null) locs[i++] = null;
         else locs[i++] = new HRegionLocation(h, server, 0);
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-client/src/main/java/org/apache/hadoop/hbase/master/RegionState.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/master/RegionState.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/master/RegionState.java
index dedc906..7598067 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/master/RegionState.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/master/RegionState.java
@@ -19,10 +19,12 @@ package org.apache.hadoop.hbase.master;
 
 import java.util.Date;
 
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.yetus.audience.InterfaceStability;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;
 
 /**
@@ -161,27 +163,27 @@ public class RegionState {
   }
 
   private final long stamp;
-  private final HRegionInfo hri;
+  private final RegionInfo hri;
   private final ServerName serverName;
   private final State state;
   // The duration of region in transition
   private long ritDuration;
 
-  public RegionState(HRegionInfo region, State state) {
+  public RegionState(RegionInfo region, State state) {
     this(region, state, System.currentTimeMillis(), null);
   }
 
-  public RegionState(HRegionInfo region,
+  public RegionState(RegionInfo region,
       State state, ServerName serverName) {
     this(region, state, System.currentTimeMillis(), serverName);
   }
 
-  public RegionState(HRegionInfo region,
+  public RegionState(RegionInfo region,
       State state, long stamp, ServerName serverName) {
     this(region, state, stamp, serverName, 0);
   }
 
-  public RegionState(HRegionInfo region, State state, long stamp, ServerName serverName,
+  public RegionState(RegionInfo region, State state, long stamp, ServerName serverName,
       long ritDuration) {
     this.hri = region;
     this.state = state;
@@ -198,7 +200,7 @@ public class RegionState {
     return stamp;
   }
 
-  public HRegionInfo getRegion() {
+  public RegionInfo getRegion() {
     return hri;
   }
 
@@ -381,7 +383,7 @@ public class RegionState {
    */
   public ClusterStatusProtos.RegionState convert() {
     ClusterStatusProtos.RegionState.Builder regionState = ClusterStatusProtos.RegionState.newBuilder();
-    regionState.setRegionInfo(HRegionInfo.convert(hri));
+    regionState.setRegionInfo(ProtobufUtil.toRegionInfo(hri));
     regionState.setState(state.convert());
     regionState.setStamp(getStamp());
     return regionState.build();
@@ -393,7 +395,7 @@ public class RegionState {
    * @return the RegionState
    */
   public static RegionState convert(ClusterStatusProtos.RegionState proto) {
-    return new RegionState(HRegionInfo.convert(proto.getRegionInfo()),
+    return new RegionState(ProtobufUtil.toRegionInfo(proto.getRegionInfo()),
       State.convert(proto.getState()), proto.getStamp(), null);
   }
 
@@ -407,7 +409,8 @@ public class RegionState {
       return false;
     }
     RegionState tmp = (RegionState)obj;
-    return tmp.hri.equals(hri) && tmp.state == state
+
+    return RegionInfo.COMPARATOR.compare(tmp.hri, hri) == 0 && tmp.state == state
       && ((serverName != null && serverName.equals(tmp.serverName))
         || (tmp.serverName == null && serverName == null));
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
index 3c9738e..1de57c5 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
@@ -25,6 +25,7 @@ import java.lang.reflect.Method;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.List;
@@ -56,7 +57,6 @@ import org.apache.hadoop.hbase.ExtendedCellBuilderFactory;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseIOException;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.ServerLoad;
@@ -1721,14 +1721,14 @@ public final class ProtobufUtil {
    * @return the retrieved region info
    * @throws IOException
    */
-  public static HRegionInfo getRegionInfo(final RpcController controller,
+  public static org.apache.hadoop.hbase.client.RegionInfo getRegionInfo(final RpcController controller,
       final AdminService.BlockingInterface admin, final byte[] regionName) throws IOException {
     try {
       GetRegionInfoRequest request =
         RequestConverter.buildGetRegionInfoRequest(regionName);
       GetRegionInfoResponse response =
         admin.getRegionInfo(controller, request);
-      return HRegionInfo.convert(response.getRegionInfo());
+      return toRegionInfo(response.getRegionInfo());
     } catch (ServiceException se) {
       throw getRemoteException(se);
     }
@@ -1787,7 +1787,7 @@ public final class ProtobufUtil {
    *
    */
   public static void warmupRegion(final RpcController controller,
-      final AdminService.BlockingInterface admin, final HRegionInfo regionInfo) throws IOException {
+      final AdminService.BlockingInterface admin, final org.apache.hadoop.hbase.client.RegionInfo regionInfo) throws IOException {
 
     try {
       WarmupRegionRequest warmupRegionRequest =
@@ -1806,7 +1806,7 @@ public final class ProtobufUtil {
    * @throws IOException
    */
   public static void openRegion(final RpcController controller,
-      final AdminService.BlockingInterface admin, ServerName server, final HRegionInfo region)
+      final AdminService.BlockingInterface admin, ServerName server, final org.apache.hadoop.hbase.client.RegionInfo region)
           throws IOException {
     OpenRegionRequest request =
       RequestConverter.buildOpenRegionRequest(server, region, null, null);
@@ -1825,7 +1825,7 @@ public final class ProtobufUtil {
    * @return a list of online region info
    * @throws IOException
    */
-  public static List<HRegionInfo> getOnlineRegions(final AdminService.BlockingInterface admin)
+  public static List<org.apache.hadoop.hbase.client.RegionInfo> getOnlineRegions(final AdminService.BlockingInterface admin)
       throws IOException {
     return getOnlineRegions(null, admin);
   }
@@ -1835,7 +1835,7 @@ public final class ProtobufUtil {
    * server using admin protocol.
    * @return a list of online region info
    */
-  public static List<HRegionInfo> getOnlineRegions(final RpcController controller,
+  public static List<org.apache.hadoop.hbase.client.RegionInfo> getOnlineRegions(final RpcController controller,
       final AdminService.BlockingInterface admin)
   throws IOException {
     GetOnlineRegionRequest request = RequestConverter.buildGetOnlineRegionRequest();
@@ -1852,13 +1852,13 @@ public final class ProtobufUtil {
    * Get the list of region info from a GetOnlineRegionResponse
    *
    * @param proto the GetOnlineRegionResponse
-   * @return the list of region info or null if <code>proto</code> is null
+   * @return the list of region info or empty if <code>proto</code> is null
    */
-  public static List<HRegionInfo> getRegionInfos(final GetOnlineRegionResponse proto) {
-    if (proto == null) return null;
-    List<HRegionInfo> regionInfos = new ArrayList<>(proto.getRegionInfoList().size());
+  public static List<org.apache.hadoop.hbase.client.RegionInfo> getRegionInfos(final GetOnlineRegionResponse proto) {
+    if (proto == null) return Collections.EMPTY_LIST;
+    List<org.apache.hadoop.hbase.client.RegionInfo> regionInfos = new ArrayList<>(proto.getRegionInfoList().size());
     for (RegionInfo regionInfo: proto.getRegionInfoList()) {
-      regionInfos.add(HRegionInfo.convert(regionInfo));
+      regionInfos.add(toRegionInfo(regionInfo));
     }
     return regionInfos;
   }
@@ -1950,7 +1950,7 @@ public final class ProtobufUtil {
     RegionSpecifierType type = regionSpecifier.getType();
     switch (type) {
       case REGION_NAME:
-        return HRegionInfo.encodeRegionName(value.toByteArray());
+        return org.apache.hadoop.hbase.client.RegionInfo.encodeRegionName(value.toByteArray());
       case ENCODED_REGION_NAME:
         return value.toStringUtf8();
       default:
@@ -2074,12 +2074,14 @@ public final class ProtobufUtil {
     return b.build();
   }
 
-  public static CompactionDescriptor toCompactionDescriptor(HRegionInfo info, byte[] family,
+  public static CompactionDescriptor toCompactionDescriptor(
+      org.apache.hadoop.hbase.client.RegionInfo info, byte[] family,
       List<Path> inputPaths, List<Path> outputPaths, Path storeDir) {
     return toCompactionDescriptor(info, null, family, inputPaths, outputPaths, storeDir);
   }
 
-  public static CompactionDescriptor toCompactionDescriptor(HRegionInfo info, byte[] regionName,
+  public static CompactionDescriptor toCompactionDescriptor(
+      org.apache.hadoop.hbase.client.RegionInfo info, byte[] regionName,
       byte[] family, List<Path> inputPaths, List<Path> outputPaths, Path storeDir) {
     // compaction descriptor contains relative paths.
     // input / output paths are relative to the store dir
@@ -2100,7 +2102,7 @@ public final class ProtobufUtil {
     return builder.build();
   }
 
-  public static FlushDescriptor toFlushDescriptor(FlushAction action, HRegionInfo hri,
+  public static FlushDescriptor toFlushDescriptor(FlushAction action, org.apache.hadoop.hbase.client.RegionInfo hri,
       long flushSeqId, Map<byte[], List<Path>> committedFiles) {
     FlushDescriptor.Builder desc = FlushDescriptor.newBuilder()
         .setAction(action)
@@ -2125,7 +2127,7 @@ public final class ProtobufUtil {
   }
 
   public static RegionEventDescriptor toRegionEventDescriptor(
-      EventType eventType, HRegionInfo hri, long seqId, ServerName server,
+      EventType eventType, org.apache.hadoop.hbase.client.RegionInfo hri, long seqId, ServerName server,
       Map<byte[], List<Path>> storeFiles) {
     final byte[] tableNameAsBytes = hri.getTable().getName();
     final byte[] encodedNameAsBytes = hri.getEncodedNameAsBytes();
@@ -3315,7 +3317,7 @@ public final class ProtobufUtil {
    * @param info the RegionInfo to convert
    * @return the converted Proto RegionInfo
    */
-  public static HBaseProtos.RegionInfo toProtoRegionInfo(final org.apache.hadoop.hbase.client.RegionInfo info) {
+  public static HBaseProtos.RegionInfo toRegionInfo(final org.apache.hadoop.hbase.client.RegionInfo info) {
     if (info == null) return null;
     HBaseProtos.RegionInfo.Builder builder = HBaseProtos.RegionInfo.newBuilder();
     builder.setTableName(ProtobufUtil.toProtoTableName(info.getTable()));