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

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

HBASE-18839 Apply RegionInfo to code base


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

Branch: refs/heads/master
Commit: a11a35a1135c431ee12534451c925727165eded5
Parents: 7f4c3b3
Author: Chia-Ping Tsai <ch...@gmail.com>
Authored: Thu Sep 28 16:16:21 2017 +0800
Committer: Chia-Ping Tsai <ch...@gmail.com>
Committed: Thu Sep 28 20:19:29 2017 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/backup/util/BackupUtils.java   |  14 +-
 .../hadoop/hbase/AsyncMetaTableAccessor.java    |  47 +--
 .../org/apache/hadoop/hbase/HRegionInfo.java    |  19 +-
 .../apache/hadoop/hbase/HRegionLocation.java    |  26 +-
 .../apache/hadoop/hbase/MetaTableAccessor.java  | 245 ++++++-------
 .../apache/hadoop/hbase/RegionLocations.java    |   5 +-
 .../apache/hadoop/hbase/client/AsyncAdmin.java  |  15 +-
 .../hadoop/hbase/client/AsyncHBaseAdmin.java    |   9 +-
 .../hbase/client/ConnectionImplementation.java  |  38 +-
 .../hbase/client/FlushRegionCallable.java       |  10 +-
 .../apache/hadoop/hbase/client/HBaseAdmin.java  | 134 ++++---
 .../hadoop/hbase/client/HRegionLocator.java     |   9 +-
 .../hadoop/hbase/client/HTableMultiplexer.java  |  15 +-
 .../hbase/client/ImmutableHRegionInfo.java      |   2 +-
 .../hadoop/hbase/client/RawAsyncHBaseAdmin.java |  64 ++--
 .../hadoop/hbase/client/RawAsyncTable.java      |  25 +-
 .../hadoop/hbase/client/RawAsyncTableImpl.java  |  18 +-
 .../client/RegionCoprocessorRpcChannelImpl.java |  22 +-
 .../apache/hadoop/hbase/client/RegionInfo.java  |  27 +-
 .../hadoop/hbase/client/RegionInfoBuilder.java  | 360 ++++++-------------
 .../hadoop/hbase/client/RegionReplicaUtil.java  |  65 ++--
 .../hadoop/hbase/client/ZooKeeperRegistry.java  |   7 +-
 .../apache/hadoop/hbase/master/RegionState.java |  23 +-
 .../hbase/shaded/protobuf/ProtobufUtil.java     |  38 +-
 .../hbase/shaded/protobuf/RequestConverter.java |  48 +--
 .../shaded/protobuf/ResponseConverter.java      |  20 +-
 .../hbase/zookeeper/MetaTableLocator.java       |  71 ++--
 .../hadoop/hbase/client/TestAsyncProcess.java   |  28 +-
 .../coprocessor/AsyncAggregationClient.java     |  28 +-
 .../apache/hadoop/hbase/coprocessor/Export.java |  22 +-
 .../example/TestRefreshHFilesEndpoint.java      |  20 +-
 .../mapreduce/TableSnapshotInputFormat.java     |  26 +-
 .../hbase/regionserver/CompactionTool.java      |  10 +-
 .../hadoop/hbase/snapshot/ExportSnapshot.java   |  23 +-
 .../hbase/mapreduce/TestImportExport.java       |  18 +-
 .../replication/TestReplicationSmallTests.java  |  25 +-
 .../hbase/snapshot/TestExportSnapshot.java      |  13 +-
 .../hbase/snapshot/TestMobExportSnapshot.java   |   4 +-
 .../hadoop/hbase/rest/RegionsResource.java      |  20 +-
 .../hbase/rsgroup/RSGroupAdminServer.java       |  43 +--
 .../hbase/rsgroup/RSGroupBasedLoadBalancer.java | 104 +++---
 .../hbase/rsgroup/RSGroupInfoManagerImpl.java   |  15 +-
 .../balancer/TestRSGroupBasedLoadBalancer.java  | 133 +++----
 .../hadoop/hbase/rsgroup/TestRSGroupsBase.java  |  22 +-
 .../hbase/tmpl/master/MasterStatusTmpl.jamon    |   6 +-
 .../hbase/tmpl/regionserver/RSStatusTmpl.jamon  |   4 +-
 .../tmpl/regionserver/RegionListTmpl.jamon      |  41 +--
 .../hadoop/hbase/RegionStateListener.java       |  11 +-
 .../hadoop/hbase/backup/HFileArchiver.java      |  18 +-
 .../hbase/client/ClientSideRegionScanner.java   |   5 +-
 .../hbase/client/TableSnapshotScanner.java      |  13 +-
 .../hbase/client/locking/LockServiceClient.java |  18 +-
 .../SplitLogManagerCoordination.java            |   8 +-
 .../ZKSplitLogManagerCoordination.java          |  16 +-
 .../coprocessor/MultiRowMutationEndpoint.java   |  16 +-
 .../favored/FavoredNodeAssignmentHelper.java    | 104 +++---
 .../hbase/favored/FavoredNodeLoadBalancer.java  |  85 ++---
 .../hbase/favored/FavoredNodesManager.java      |  44 +--
 .../hadoop/hbase/favored/FavoredNodesPlan.java  |  10 +-
 .../hbase/favored/FavoredNodesPromoter.java     |   8 +-
 .../org/apache/hadoop/hbase/io/HFileLink.java   |  19 +-
 .../hadoop/hbase/master/AssignmentListener.java |   9 +-
 .../master/AssignmentVerificationReport.java    |  38 +-
 .../hadoop/hbase/master/CatalogJanitor.java     |  56 ++-
 .../org/apache/hadoop/hbase/master/HMaster.java |  58 +--
 .../hadoop/hbase/master/LoadBalancer.java       |  32 +-
 .../hadoop/hbase/master/MasterFileSystem.java   |  13 +-
 .../hbase/master/MasterMetaBootstrap.java       |  15 +-
 .../hadoop/hbase/master/MasterRpcServices.java  |  54 +--
 .../hadoop/hbase/master/MasterServices.java     |  11 +-
 .../hadoop/hbase/master/MasterWalManager.java   |   6 +-
 .../hbase/master/RegionPlacementMaintainer.java |  67 ++--
 .../apache/hadoop/hbase/master/RegionPlan.java  |  10 +-
 .../hadoop/hbase/master/ServerManager.java      |  32 +-
 .../SnapshotOfRegionAssignmentFromMeta.java     |  64 ++--
 .../hadoop/hbase/master/SplitLogManager.java    |  12 +-
 .../master/assignment/AssignProcedure.java      |  16 +-
 .../master/assignment/AssignmentManager.java    | 217 +++++------
 .../assignment/GCMergedRegionsProcedure.java    |  33 +-
 .../master/assignment/GCRegionProcedure.java    |  16 +-
 .../assignment/MergeTableRegionsProcedure.java  |  89 ++---
 .../master/assignment/MoveRegionProcedure.java  |   9 +-
 .../master/assignment/RegionStateStore.java     |  34 +-
 .../hbase/master/assignment/RegionStates.java   | 168 ++++-----
 .../assignment/RegionTransitionProcedure.java   |  15 +-
 .../assignment/SplitTableRegionProcedure.java   |  84 +++--
 .../master/assignment/UnassignProcedure.java    |  22 +-
 .../hadoop/hbase/master/assignment/Util.java    |   9 +-
 .../hbase/master/balancer/BaseLoadBalancer.java | 118 +++---
 .../hbase/master/balancer/ClusterLoadState.java |  16 +-
 .../balancer/FavoredStochasticBalancer.java     | 102 +++---
 .../master/balancer/RegionInfoComparator.java   |   6 +-
 .../master/balancer/RegionLocationFinder.java   |  38 +-
 .../master/balancer/SimpleLoadBalancer.java     |  42 +--
 .../master/balancer/StochasticLoadBalancer.java |  14 +-
 .../hbase/master/locking/LockManager.java       |  15 +-
 .../hbase/master/locking/LockProcedure.java     |  27 +-
 .../normalizer/MergeNormalizationPlan.java      |  19 +-
 .../master/normalizer/RegionNormalizer.java     |   8 +-
 .../normalizer/SimpleRegionNormalizer.java      |  17 +-
 .../normalizer/SplitNormalizationPlan.java      |  19 +-
 .../AbstractStateMachineRegionProcedure.java    |  21 +-
 .../procedure/AddColumnFamilyProcedure.java     |   9 +-
 .../procedure/CloneSnapshotProcedure.java       |  38 +-
 .../master/procedure/CreateTableProcedure.java  |  66 ++--
 .../procedure/DeleteColumnFamilyProcedure.java  |  11 +-
 .../master/procedure/DeleteTableProcedure.java  |  23 +-
 .../procedure/MasterDDLOperationHelper.java     |  10 +-
 .../procedure/MasterProcedureScheduler.java     |  19 +-
 .../master/procedure/ModifyTableProcedure.java  |  11 +-
 .../master/procedure/ProcedureSyncWait.java     |  10 +-
 .../master/procedure/RSProcedureDispatcher.java |  26 +-
 .../master/procedure/RecoverMetaProcedure.java  |  24 +-
 .../procedure/RestoreSnapshotProcedure.java     |  39 +-
 .../master/procedure/ServerCrashProcedure.java  |  31 +-
 .../procedure/TruncateTableProcedure.java       |  29 +-
 .../snapshot/DisabledTableSnapshotHandler.java  |  21 +-
 .../snapshot/EnabledTableSnapshotHandler.java   |  20 +-
 .../master/snapshot/MasterSnapshotVerifier.java |  31 +-
 .../master/snapshot/TakeSnapshotHandler.java    |  17 +-
 .../org/apache/hadoop/hbase/mob/MobUtils.java   |  26 +-
 .../hbase/namespace/NamespaceAuditor.java       |  12 +-
 .../hbase/namespace/NamespaceStateManager.java  |  16 +-
 .../flush/MasterFlushTableProcedureManager.java |  15 +-
 .../quotas/FileSystemUtilizationChore.java      |  12 +-
 .../hadoop/hbase/quotas/MasterQuotaManager.java |  32 +-
 .../quotas/NamespaceQuotaSnapshotStore.java     |  24 +-
 .../hadoop/hbase/quotas/QuotaObserverChore.java |  15 +-
 .../hadoop/hbase/quotas/QuotaSnapshotStore.java |  11 +-
 .../hbase/quotas/TableQuotaSnapshotStore.java   |  24 +-
 .../hadoop/hbase/regionserver/HRegion.java      |  64 ++--
 .../hbase/regionserver/HRegionFileSystem.java   |  74 ++--
 .../hbase/regionserver/HRegionServer.java       |  64 ++--
 .../hadoop/hbase/regionserver/HStore.java       |  12 +-
 .../regionserver/MetricsRegionWrapperImpl.java  |   4 +-
 .../hbase/regionserver/RSRpcServices.java       | 101 +++---
 .../hadoop/hbase/regionserver/Region.java       |   8 +-
 .../hbase/regionserver/RegionMergeRequest.java  |  15 +-
 .../hbase/regionserver/RegionScanner.java       |   8 +-
 .../regionserver/RegionServerServices.java      |  20 +-
 .../regionserver/RegionServicesForStores.java   |   6 +-
 .../hbase/regionserver/RegionSplitPolicy.java   |   2 +-
 .../hadoop/hbase/regionserver/SplitRequest.java |  19 +-
 .../regionserver/handler/CloseMetaHandler.java  |   6 +-
 .../handler/CloseRegionHandler.java             |  15 +-
 .../regionserver/handler/OpenMetaHandler.java   |   6 +-
 .../handler/OpenPriorityRegionHandler.java      |   6 +-
 .../regionserver/handler/OpenRegionHandler.java |  17 +-
 .../hbase/regionserver/wal/AbstractFSWAL.java   |  19 +-
 .../hbase/regionserver/wal/AsyncFSWAL.java      |  24 +-
 .../hadoop/hbase/regionserver/wal/FSHLog.java   |  20 +-
 .../hbase/regionserver/wal/FSWALEntry.java      |  20 +-
 .../regionserver/wal/WALCoprocessorHost.java    |   6 +-
 .../hadoop/hbase/regionserver/wal/WALUtil.java  |  24 +-
 .../RegionReplicaReplicationEndpoint.java       |  12 +-
 .../hbase/snapshot/RestoreSnapshotHelper.java   | 129 +++----
 .../hadoop/hbase/snapshot/SnapshotInfo.java     |  30 +-
 .../hadoop/hbase/snapshot/SnapshotManifest.java |  16 +-
 .../hbase/snapshot/SnapshotManifestV1.java      |  18 +-
 .../hbase/snapshot/SnapshotManifestV2.java      |  23 +-
 .../hbase/snapshot/SnapshotReferenceUtil.java   |  23 +-
 .../org/apache/hadoop/hbase/tool/Canary.java    |  68 ++--
 .../org/apache/hadoop/hbase/util/HBaseFsck.java | 184 +++++-----
 .../hadoop/hbase/util/HBaseFsckRepair.java      |  40 +--
 .../hadoop/hbase/util/HFileArchiveUtil.java     |  14 +-
 .../hadoop/hbase/util/ModifyRegionUtils.java    |  68 ++--
 .../apache/hadoop/hbase/util/RegionMover.java   |  60 ++--
 .../hbase/util/ServerRegionReplicaUtil.java     |  13 +-
 .../hadoop/hbase/wal/DisabledWALProvider.java   |  12 +-
 .../java/org/apache/hadoop/hbase/wal/WAL.java   |  16 +-
 .../org/apache/hadoop/hbase/wal/WALEdit.java    |  24 +-
 .../resources/hbase-webapps/master/table.jsp    | 111 +++---
 .../hbase-webapps/regionserver/region.jsp       |   9 +-
 .../hadoop/hbase/HBaseTestingUtility.java       | 244 ++-----------
 .../hadoop/hbase/MockRegionServerServices.java  |  18 +-
 .../hadoop/hbase/TestGlobalMemStoreSize.java    |  10 +-
 .../org/apache/hadoop/hbase/TestIOFencing.java  |   7 +-
 .../hadoop/hbase/TestMetaTableAccessor.java     | 207 ++++++++---
 .../hbase/TestMetaTableAccessorNoCluster.java   |  27 +-
 .../hbase/TestPartialResultsFromClientSide.java |   5 +-
 .../hadoop/hbase/TestRegionRebalancing.java     |   6 +-
 .../hbase/TestStochasticBalancerJmxMetrics.java |  22 +-
 .../org/apache/hadoop/hbase/TestZooKeeper.java  |   5 +-
 .../hbase/client/HConnectionTestingUtility.java |   4 +-
 .../apache/hadoop/hbase/client/TestAdmin1.java  |  23 +-
 .../apache/hadoop/hbase/client/TestAdmin2.java  |  36 +-
 .../hbase/client/TestAsyncClusterAdminApi.java  |  11 +-
 .../hbase/client/TestAsyncRegionAdminApi.java   |  48 +--
 .../client/TestBlockEvictionFromClient.java     |   7 +-
 .../hbase/client/TestMetaWithReplicas.java      |  11 +-
 .../hbase/client/TestMvccConsistentScanner.java |   3 +-
 .../coprocessor/TestCoprocessorInterface.java   |  13 +-
 .../hbase/coprocessor/TestMasterObserver.java   |  22 +-
 .../TestRegionObserverInterface.java            |   4 +-
 .../TestRegionObserverScannerOpenHook.java      |   3 +-
 .../TestFavoredNodeAssignmentHelper.java        | 108 +++---
 .../hbase/master/MockNoopMasterServices.java    |  10 +-
 .../hadoop/hbase/master/MockRegionServer.java   |  13 +-
 .../hbase/master/TestAssignmentListener.java    |   8 +-
 .../TestCatalogJanitorInMemoryStates.java       |  51 +--
 .../master/TestDistributedLogSplitting.java     | 138 +++----
 .../apache/hadoop/hbase/master/TestMaster.java  |  23 +-
 .../TestMasterOperationsForRegionReplicas.java  |  55 ++-
 .../hbase/master/TestMasterTransitions.java     |  16 +-
 .../hbase/master/TestRegionPlacement.java       |  14 +-
 .../hbase/master/TestRegionPlacement2.java      |  21 +-
 .../hadoop/hbase/master/TestRestartCluster.java |  13 +-
 .../hadoop/hbase/master/TestRollingRestart.java |   7 +-
 .../hadoop/hbase/master/TestWarmupRegion.java   |   6 +-
 .../assignment/AssignmentTestingUtil.java       |  22 +-
 .../master/assignment/MockMasterServices.java   |  16 +-
 .../assignment/TestAssignmentManager.java       |  82 +++--
 .../assignment/TestAssignmentOnRSCrash.java     |  25 +-
 .../TestMergeTableRegionsProcedure.java         |  28 +-
 .../master/assignment/TestRegionStates.java     |  50 +--
 .../TestSplitTableRegionProcedure.java          |  20 +-
 .../hbase/master/balancer/BalancerTestBase.java | 100 +++---
 .../LoadBalancerPerformanceEvaluation.java      |  37 +-
 .../master/balancer/TestBaseLoadBalancer.java   | 149 ++++----
 .../balancer/TestDefaultLoadBalancer.java       |  32 +-
 .../TestFavoredStochasticBalancerPickers.java   |  20 +-
 .../TestFavoredStochasticLoadBalancer.java      |  62 ++--
 .../balancer/TestRegionLocationFinder.java      |   8 +-
 .../balancer/TestStochasticLoadBalancer.java    |  38 +-
 .../hbase/master/locking/TestLockProcedure.java |  27 +-
 .../normalizer/TestSimpleRegionNormalizer.java  | 228 ++++++++----
 .../TestSimpleRegionNormalizerOnCluster.java    |  30 +-
 .../MasterProcedureTestingUtility.java          |  14 +-
 .../procedure/TestCreateTableProcedure.java     |  18 +-
 .../TestDeleteColumnFamilyProcedure.java        |   4 +-
 .../procedure/TestDeleteTableProcedure.java     |   8 +-
 .../TestMasterFailoverWithProcedures.java       |  21 +-
 .../procedure/TestMasterProcedureWalLease.java  |   6 +-
 .../procedure/TestModifyTableProcedure.java     |  10 +-
 .../master/procedure/TestProcedureAdmin.java    |   6 +-
 .../procedure/TestSafemodeBringsDownMaster.java |   5 +-
 .../procedure/TestTruncateTableProcedure.java   |  10 +-
 .../master/snapshot/TestSnapshotFileCache.java  |  22 +-
 .../quotas/TestFileSystemUtilizationChore.java  |  29 +-
 .../TestNamespaceQuotaViolationStore.java       |  59 ++-
 .../hbase/quotas/TestQuotaObserverChore.java    |  20 +-
 .../TestQuotaObserverChoreRegionReports.java    |  10 +-
 .../hbase/quotas/TestQuotaStatusRPCs.java       |  14 +-
 .../hadoop/hbase/quotas/TestRegionSizeUse.java  |  10 +-
 .../hadoop/hbase/quotas/TestSpaceQuotas.java    |  10 +-
 .../quotas/TestTableQuotaViolationStore.java    |  63 +++-
 .../hbase/quotas/TestTablesWithQuotas.java      |   4 +-
 .../hbase/regionserver/TestAtomicOperation.java |  25 +-
 .../hbase/regionserver/TestDefaultMemStore.java |   4 +-
 .../TestEndToEndSplitTransaction.java           |  42 +--
 .../hadoop/hbase/regionserver/TestHRegion.java  |   3 +-
 .../regionserver/TestHRegionFileSystem.java     |  14 +-
 .../hbase/regionserver/TestHRegionInfo.java     |   8 +-
 .../hbase/regionserver/TestMobStoreScanner.java |   4 +-
 .../TestRSKilledWhenInitializing.java           |  12 +-
 .../hbase/regionserver/TestRSStatusServlet.java |  45 +--
 .../regionserver/TestRegionInfoBuilder.java     |   4 +-
 .../TestRegionMergeTransactionOnCluster.java    |  64 ++--
 .../regionserver/TestRegionServerAbort.java     |  15 +-
 .../TestRegionServerRegionSpaceUseReport.java   |  44 ++-
 .../TestScannerHeartbeatMessages.java           |  16 +-
 .../TestSplitTransactionOnCluster.java          |  32 +-
 .../regionserver/wal/AbstractTestFSWAL.java     |  23 +-
 .../wal/AbstractTestLogRolling.java             |   5 +-
 .../regionserver/wal/TestAsyncLogRolling.java   |   6 +-
 .../hbase/regionserver/wal/TestLogRolling.java  |   6 +-
 .../replication/TestReplicationEndpoint.java    |  20 +-
 .../replication/TestSerialReplication.java      |  20 +-
 .../hbase/snapshot/SnapshotTestingUtils.java    |  53 +--
 .../TestLoadIncrementalHFilesSplitRecovery.java |  26 +-
 .../hadoop/hbase/util/BaseTestHBaseFsck.java    |  32 +-
 .../hadoop/hbase/util/TestHBaseFsckOneRS.java   |  70 ++--
 .../hbase/util/TestHBaseFsckReplicas.java       |  71 ++--
 .../hadoop/hbase/util/TestHBaseFsckTwoRS.java   |  46 ++-
 .../util/hbck/OfflineMetaRebuildTestCore.java   |  18 +-
 .../apache/hadoop/hbase/wal/FaultyFSLog.java    |  12 +-
 .../hbase/wal/WALPerformanceEvaluation.java     |  29 +-
 .../hadoop/hbase/thrift/ThriftServerRunner.java |  64 ++--
 278 files changed, 4817 insertions(+), 4611 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java
index 631183d..9d87612 100644
--- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java
@@ -43,7 +43,6 @@ import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
@@ -53,15 +52,16 @@ import org.apache.hadoop.hbase.backup.HBackupFileSystem;
 import org.apache.hadoop.hbase.backup.RestoreRequest;
 import org.apache.hadoop.hbase.backup.impl.BackupManifest;
 import org.apache.hadoop.hbase.backup.impl.BackupManifest.BackupImage;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.TableDescriptor;
-import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
 import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSTableDescriptors;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * A collection for methods used by multiple classes to backup HBase tables.
@@ -150,11 +150,11 @@ public final class BackupUtils {
       LOG.debug("Attempting to copy table info for:" + table + " target: " + target
           + " descriptor: " + orig);
       LOG.debug("Finished copying tableinfo.");
-      List<HRegionInfo> regions = null;
+      List<RegionInfo> regions = null;
       regions = MetaTableAccessor.getTableRegions(conn, table);
       // For each region, write the region info to disk
       LOG.debug("Starting to write region info for table " + table);
-      for (HRegionInfo regionInfo : regions) {
+      for (RegionInfo regionInfo : regions) {
         Path regionDir =
             HRegion.getRegionDir(new Path(backupInfo.getTableBackupDir(table)), regionInfo);
         regionDir = new Path(backupInfo.getTableBackupDir(table), regionDir.getName());
@@ -168,8 +168,8 @@ public final class BackupUtils {
    * Write the .regioninfo file on-disk.
    */
   public static void writeRegioninfoOnFilesystem(final Configuration conf, final FileSystem fs,
-      final Path regionInfoDir, HRegionInfo regionInfo) throws IOException {
-    final byte[] content = regionInfo.toDelimitedByteArray();
+      final Path regionInfoDir, RegionInfo regionInfo) throws IOException {
+    final byte[] content = RegionInfo.toDelimitedByteArray(regionInfo);
     Path regionInfoFile = new Path(regionInfoDir, "." + HConstants.REGIONINFO_QUALIFIER_STR);
     // First check to get the permissions
     FsPermission perms = FSUtils.getFilePermissions(fs, conf, HConstants.DATA_FILE_UMASK_KEY);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-client/src/main/java/org/apache/hadoop/hbase/AsyncMetaTableAccessor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/AsyncMetaTableAccessor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/AsyncMetaTableAccessor.java
index 678dd91..6f41bd0 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/AsyncMetaTableAccessor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/AsyncMetaTableAccessor.java
@@ -38,20 +38,21 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.MetaTableAccessor.CollectingVisitor;
 import org.apache.hadoop.hbase.MetaTableAccessor.QueryType;
 import org.apache.hadoop.hbase.MetaTableAccessor.Visitor;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Consistency;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.RawAsyncTable;
 import org.apache.hadoop.hbase.client.RawScanResultConsumer;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.TableState;
 import org.apache.hadoop.hbase.client.Scan.ReadType;
+import org.apache.hadoop.hbase.client.TableState;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.Pair;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * The asynchronous meta table accessor. Used to read/write region and assignment information store
@@ -112,7 +113,7 @@ public class AsyncMetaTableAccessor {
       RawAsyncTable metaTable, byte[] regionName) {
     CompletableFuture<Optional<HRegionLocation>> future = new CompletableFuture<>();
     try {
-      HRegionInfo parsedRegionInfo = MetaTableAccessor.parseRegionInfoFromRegionName(regionName);
+      RegionInfo parsedRegionInfo = MetaTableAccessor.parseRegionInfoFromRegionName(regionName);
       metaTable.get(
         new Get(MetaTableAccessor.getMetaKeyForRegion(parsedRegionInfo))
             .addFamily(HConstants.CATALOG_FAMILY)).whenComplete(
@@ -151,14 +152,14 @@ public class AsyncMetaTableAccessor {
             results
                 .stream()
                 .filter(result -> !result.isEmpty())
-                .filter(result -> MetaTableAccessor.getHRegionInfo(result) != null)
+                .filter(result -> MetaTableAccessor.getRegionInfo(result) != null)
                 .forEach(
                   result -> {
                     getRegionLocations(result).ifPresent(
                       locations -> {
                         for (HRegionLocation location : locations.getRegionLocations()) {
                           if (location != null
-                              && encodedRegionNameStr.equals(location.getRegionInfo()
+                              && encodedRegionNameStr.equals(location.getRegion()
                                   .getEncodedName())) {
                             future.complete(Optional.of(location));
                             return;
@@ -218,27 +219,27 @@ public class AsyncMetaTableAccessor {
    * @return the list of regioninfos and server. The return value will be wrapped by a
    *         {@link CompletableFuture}.
    */
-  private static CompletableFuture<List<Pair<HRegionInfo, ServerName>>> getTableRegionsAndLocations(
+  private static CompletableFuture<List<Pair<RegionInfo, ServerName>>> getTableRegionsAndLocations(
       RawAsyncTable metaTable, final Optional<TableName> tableName,
       final boolean excludeOfflinedSplitParents) {
-    CompletableFuture<List<Pair<HRegionInfo, ServerName>>> future = new CompletableFuture<>();
+    CompletableFuture<List<Pair<RegionInfo, ServerName>>> future = new CompletableFuture<>();
     if (tableName.filter((t) -> t.equals(TableName.META_TABLE_NAME)).isPresent()) {
       future.completeExceptionally(new IOException(
           "This method can't be used to locate meta regions;" + " use MetaTableLocator instead"));
     }
 
-    // Make a version of CollectingVisitor that collects HRegionInfo and ServerAddress
-    CollectingVisitor<Pair<HRegionInfo, ServerName>> visitor = new CollectingVisitor<Pair<HRegionInfo, ServerName>>() {
+    // Make a version of CollectingVisitor that collects RegionInfo and ServerAddress
+    CollectingVisitor<Pair<RegionInfo, ServerName>> visitor = new CollectingVisitor<Pair<RegionInfo, ServerName>>() {
       private Optional<RegionLocations> current = null;
 
       @Override
       public boolean visit(Result r) throws IOException {
         current = getRegionLocations(r);
-        if (!current.isPresent() || current.get().getRegionLocation().getRegionInfo() == null) {
-          LOG.warn("No serialized HRegionInfo in " + r);
+        if (!current.isPresent() || current.get().getRegionLocation().getRegion() == null) {
+          LOG.warn("No serialized RegionInfo in " + r);
           return true;
         }
-        HRegionInfo hri = current.get().getRegionLocation().getRegionInfo();
+        RegionInfo hri = current.get().getRegionLocation().getRegion();
         if (excludeOfflinedSplitParents && hri.isSplitParent()) return true;
         // Else call super and add this Result to the collection.
         return super.visit(r);
@@ -251,7 +252,7 @@ public class AsyncMetaTableAccessor {
         }
         for (HRegionLocation loc : current.get().getRegionLocations()) {
           if (loc != null) {
-            this.results.add(new Pair<HRegionInfo, ServerName>(loc.getRegionInfo(), loc
+            this.results.add(new Pair<RegionInfo, ServerName>(loc.getRegionInfo(), loc
                 .getServerName()));
           }
         }
@@ -381,7 +382,7 @@ public class AsyncMetaTableAccessor {
    */
   private static Optional<RegionLocations> getRegionLocations(final Result r) {
     if (r == null) return Optional.empty();
-    Optional<HRegionInfo> regionInfo = getHRegionInfo(r, getRegionInfoColumn());
+    Optional<RegionInfo> regionInfo = getHRegionInfo(r, getRegionInfoColumn());
     if (!regionInfo.isPresent()) return Optional.empty();
 
     List<HRegionLocation> locations = new ArrayList<HRegionLocation>(1);
@@ -427,11 +428,11 @@ public class AsyncMetaTableAccessor {
    * @param replicaId the replicaId for the HRegionLocation
    * @return HRegionLocation parsed from the given meta row Result for the given replicaId
    */
-  private static HRegionLocation getRegionLocation(final Result r, final HRegionInfo regionInfo,
+  private static HRegionLocation getRegionLocation(final Result r, final RegionInfo regionInfo,
       final int replicaId) {
     Optional<ServerName> serverName = getServerName(r, replicaId);
     long seqNum = getSeqNumDuringOpen(r, replicaId);
-    HRegionInfo replicaInfo = RegionReplicaUtil.getRegionInfoForReplica(regionInfo, replicaId);
+    RegionInfo replicaInfo = RegionReplicaUtil.getRegionInfoForReplica(regionInfo, replicaId);
     return new HRegionLocation(replicaInfo, serverName.orElse(null), seqNum);
   }
 
@@ -521,16 +522,16 @@ public class AsyncMetaTableAccessor {
   }
 
   /**
-   * Returns the HRegionInfo object from the column {@link HConstants#CATALOG_FAMILY} and
+   * Returns the RegionInfo object from the column {@link HConstants#CATALOG_FAMILY} and
    * <code>qualifier</code> of the catalog table result.
    * @param r a Result object from the catalog table scan
    * @param qualifier Column family qualifier
-   * @return An HRegionInfo instance.
+   * @return An RegionInfo instance.
    */
-  private static Optional<HRegionInfo> getHRegionInfo(final Result r, byte[] qualifier) {
+  private static Optional<RegionInfo> getHRegionInfo(final Result r, byte[] qualifier) {
     Cell cell = r.getColumnLatestCell(getCatalogFamily(), qualifier);
     if (cell == null) return Optional.empty();
-    return Optional.ofNullable(HRegionInfo.parseFromOrNull(cell.getValueArray(),
+    return Optional.ofNullable(RegionInfo.parseFromOrNull(cell.getValueArray(),
       cell.getValueOffset(), cell.getValueLength()));
   }
 
@@ -575,7 +576,7 @@ public class AsyncMetaTableAccessor {
     return replicaId == 0
       ? HConstants.SERVER_QUALIFIER
       : Bytes.toBytes(HConstants.SERVER_QUALIFIER_STR + META_REPLICA_ID_DELIMITER
-      + String.format(HRegionInfo.REPLICA_ID_FORMAT, replicaId));
+      + String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId));
   }
 
   /**
@@ -587,7 +588,7 @@ public class AsyncMetaTableAccessor {
     return replicaId == 0
       ? HConstants.STARTCODE_QUALIFIER
       : Bytes.toBytes(HConstants.STARTCODE_QUALIFIER_STR + META_REPLICA_ID_DELIMITER
-      + String.format(HRegionInfo.REPLICA_ID_FORMAT, replicaId));
+      + String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId));
   }
 
   /**
@@ -599,7 +600,7 @@ public class AsyncMetaTableAccessor {
     return replicaId == 0
       ? HConstants.SEQNUM_QUALIFIER
       : Bytes.toBytes(HConstants.SEQNUM_QUALIFIER_STR + META_REPLICA_ID_DELIMITER
-      + String.format(HRegionInfo.REPLICA_ID_FORMAT, replicaId));
+      + String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId));
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
index cc88733..f2cb821 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
@@ -28,17 +28,18 @@ import java.util.stream.Collectors;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.client.RegionInfoBuilder;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.KeyValue.KVComparator;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.RegionInfoDisplay;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.master.RegionState;
-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.client.RegionInfoDisplay;
 import org.apache.hadoop.io.DataInputBuffer;
+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 of a table, an
@@ -308,7 +309,7 @@ public class HRegionInfo implements RegionInfo, Comparable<HRegionInfo> {
    *
    * @param other
    */
-  public HRegionInfo(HRegionInfo other) {
+  public HRegionInfo(RegionInfo other) {
     super();
     this.endKey = other.getEndKey();
     this.offLine = other.isOffline();
@@ -710,13 +711,13 @@ public class HRegionInfo implements RegionInfo, Comparable<HRegionInfo> {
    * @param info the HRegionInfo to convert
    * @return the converted RegionInfo
    * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
-   *             Use toProtoRegionInfo(org.apache.hadoop.hbase.client.RegionInfo)
+   *             Use toRegionInfo(org.apache.hadoop.hbase.client.RegionInfo)
    *             in org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.
    */
   @Deprecated
   @InterfaceAudience.Private
   public static HBaseProtos.RegionInfo convert(final HRegionInfo info) {
-    return ProtobufUtil.toProtoRegionInfo(info);
+    return ProtobufUtil.toRegionInfo(info);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionLocation.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionLocation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionLocation.java
index c6c8af5..fd679bd 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionLocation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionLocation.java
@@ -18,11 +18,13 @@
  */
 package org.apache.hadoop.hbase;
 
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.client.ImmutableHRegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.util.Addressing;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
- * Data structure to hold HRegionInfo and the address for the hosting
+ * Data structure to hold RegionInfo and the address for the hosting
  * HRegionServer.  Immutable.  Comparable, but we compare the 'location' only:
  * i.e. the hostname and port, and *not* the regioninfo.  This means two
  * instances are the same if they refer to the same 'location' (the same
@@ -36,15 +38,15 @@ import org.apache.hadoop.hbase.util.Addressing;
  */
 @InterfaceAudience.Public
 public class HRegionLocation implements Comparable<HRegionLocation> {
-  private final HRegionInfo regionInfo;
+  private final RegionInfo regionInfo;
   private final ServerName serverName;
   private final long seqNum;
 
-  public HRegionLocation(HRegionInfo regionInfo, ServerName serverName) {
+  public HRegionLocation(RegionInfo regionInfo, ServerName serverName) {
     this(regionInfo, serverName, HConstants.NO_SEQNUM);
   }
 
-  public HRegionLocation(HRegionInfo regionInfo, ServerName serverName, long seqNum) {
+  public HRegionLocation(RegionInfo regionInfo, ServerName serverName, long seqNum) {
     this.regionInfo = regionInfo;
     this.serverName = serverName;
     this.seqNum = seqNum;
@@ -84,8 +86,20 @@ public class HRegionLocation implements Comparable<HRegionLocation> {
     return this.serverName.hashCode();
   }
 
-  /** @return HRegionInfo */
+  /**
+   *
+   * @return Immutable HRegionInfo
+   * @deprecated Since 2.0.0. Will remove in 3.0.0. Use {@link #getRegion()}} instead.
+   */
+  @Deprecated
   public HRegionInfo getRegionInfo(){
+    return regionInfo == null ? null : new ImmutableHRegionInfo(regionInfo);
+  }
+
+  /**
+   * @return regionInfo
+   */
+  public RegionInfo getRegion(){
     return regionInfo;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
index 47fffa2..8b6cf66 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
@@ -22,6 +22,7 @@ import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.LinkedHashMap;
 import java.util.List;
@@ -36,7 +37,6 @@ import java.util.regex.Pattern;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Consistency;
@@ -44,6 +44,8 @@ import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
+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.RegionServerCallable;
@@ -66,6 +68,7 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.ExceptionUtil;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.PairOfSameType;
+import org.apache.yetus.audience.InterfaceAudience;
 
 import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
 
@@ -152,9 +155,9 @@ public class MetaTableAccessor {
   static {
     // Copy the prefix from FIRST_META_REGIONINFO into META_REGION_PREFIX.
     // FIRST_META_REGIONINFO == 'hbase:meta,,1'.  META_REGION_PREFIX == 'hbase:meta,'
-    int len = HRegionInfo.FIRST_META_REGIONINFO.getRegionName().length - 2;
+    int len = RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName().length - 2;
     META_REGION_PREFIX = new byte [len];
-    System.arraycopy(HRegionInfo.FIRST_META_REGIONINFO.getRegionName(), 0,
+    System.arraycopy(RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName(), 0,
       META_REGION_PREFIX, 0, len);
   }
 
@@ -168,9 +171,9 @@ public class MetaTableAccessor {
    * @deprecated use {@link #getTableRegionsAndLocations}, region can have multiple locations
    */
   @Deprecated
-  public static NavigableMap<HRegionInfo, ServerName> allTableRegions(
+  public static NavigableMap<RegionInfo, ServerName> allTableRegions(
       Connection connection, final TableName tableName) throws IOException {
-    final NavigableMap<HRegionInfo, ServerName> regions = new TreeMap<>();
+    final NavigableMap<RegionInfo, ServerName> regions = new TreeMap<>();
     Visitor visitor = new TableVisitorBase(tableName) {
       @Override
       public boolean visitInternal(Result result) throws IOException {
@@ -178,7 +181,7 @@ public class MetaTableAccessor {
         if (locations == null) return true;
         for (HRegionLocation loc : locations.getRegionLocations()) {
           if (loc != null) {
-            HRegionInfo regionInfo = loc.getRegionInfo();
+            RegionInfo regionInfo = loc.getRegionInfo();
             regions.put(regionInfo, loc.getServerName());
           }
         }
@@ -300,12 +303,12 @@ public class MetaTableAccessor {
    * Gets the region info and assignment for the specified region.
    * @param connection connection we're using
    * @param regionName Region to lookup.
-   * @return Location and HRegionInfo for <code>regionName</code>
+   * @return Location and RegionInfo for <code>regionName</code>
    * @throws IOException
    * @deprecated use {@link #getRegionLocation(Connection, byte[])} instead
    */
   @Deprecated
-  public static Pair<HRegionInfo, ServerName> getRegion(Connection connection, byte [] regionName)
+  public static Pair<RegionInfo, ServerName> getRegion(Connection connection, byte [] regionName)
     throws IOException {
     HRegionLocation location = getRegionLocation(connection, regionName);
     return location == null
@@ -323,7 +326,7 @@ public class MetaTableAccessor {
   public static HRegionLocation getRegionLocation(Connection connection,
                                                   byte[] regionName) throws IOException {
     byte[] row = regionName;
-    HRegionInfo parsedInfo = null;
+    RegionInfo parsedInfo = null;
     try {
       parsedInfo = parseRegionInfoFromRegionName(regionName);
       row = getMetaKeyForRegion(parsedInfo);
@@ -347,7 +350,7 @@ public class MetaTableAccessor {
    * @throws IOException
    */
   public static HRegionLocation getRegionLocation(Connection connection,
-                                                  HRegionInfo regionInfo) throws IOException {
+                                                  RegionInfo regionInfo) throws IOException {
     byte[] row = getMetaKeyForRegion(regionInfo);
     Get get = new Get(row);
     get.addFamily(HConstants.CATALOG_FAMILY);
@@ -356,7 +359,7 @@ public class MetaTableAccessor {
   }
 
   /** Returns the row key to use for this regionInfo */
-  public static byte[] getMetaKeyForRegion(HRegionInfo regionInfo) {
+  public static byte[] getMetaKeyForRegion(RegionInfo regionInfo) {
     return RegionReplicaUtil.getRegionInfoForDefaultReplica(regionInfo).getRegionName();
   }
 
@@ -364,12 +367,17 @@ public class MetaTableAccessor {
    * is stored in the name, so the returned object should only be used for the fields
    * in the regionName.
    */
-  public static HRegionInfo parseRegionInfoFromRegionName(byte[] regionName) throws IOException {
-    byte[][] fields = HRegionInfo.parseRegionName(regionName);
+  public static RegionInfo parseRegionInfoFromRegionName(byte[] regionName) throws IOException {
+    byte[][] fields = RegionInfo.parseRegionName(regionName);
     long regionId = Long.parseLong(Bytes.toString(fields[2]));
     int replicaId = fields.length > 3 ? Integer.parseInt(Bytes.toString(fields[3]), 16) : 0;
-    return new HRegionInfo(TableName.valueOf(fields[0]), fields[1], fields[1], false, regionId,
-        replicaId);
+    return RegionInfoBuilder.newBuilder(TableName.valueOf(fields[0]))
+              .setStartKey(fields[1])
+              .setEndKey(fields[2])
+              .setSplit(false)
+              .setRegionId(regionId)
+              .setReplicaId(replicaId)
+              .build();
   }
 
   /**
@@ -392,11 +400,11 @@ public class MetaTableAccessor {
    * @throws IOException
    */
   @Nullable
-  public static Pair<HRegionInfo, HRegionInfo> getRegionsFromMergeQualifier(
+  public static Pair<RegionInfo, RegionInfo> getRegionsFromMergeQualifier(
       Connection connection, byte[] regionName) throws IOException {
     Result result = getRegionResult(connection, regionName);
-    HRegionInfo mergeA = getHRegionInfo(result, HConstants.MERGEA_QUALIFIER);
-    HRegionInfo mergeB = getHRegionInfo(result, HConstants.MERGEB_QUALIFIER);
+    RegionInfo mergeA = getRegionInfo(result, HConstants.MERGEA_QUALIFIER);
+    RegionInfo mergeB = getRegionInfo(result, HConstants.MERGEB_QUALIFIER);
     if (mergeA == null && mergeB == null) {
       return null;
     }
@@ -429,15 +437,15 @@ public class MetaTableAccessor {
    * @throws IOException
    */
   @VisibleForTesting
-  public static List<HRegionInfo> getAllRegions(Connection connection,
+  public static List<RegionInfo> getAllRegions(Connection connection,
       boolean excludeOfflinedSplitParents)
       throws IOException {
-    List<Pair<HRegionInfo, ServerName>> result;
+    List<Pair<RegionInfo, ServerName>> result;
 
     result = getTableRegionsAndLocations(connection, null,
         excludeOfflinedSplitParents);
 
-    return getListOfHRegionInfos(result);
+    return getListOfRegionInfos(result);
 
   }
 
@@ -446,10 +454,10 @@ public class MetaTableAccessor {
    * to get meta table regions, use methods in MetaTableLocator instead.
    * @param connection connection we're using
    * @param tableName table we're looking for
-   * @return Ordered list of {@link HRegionInfo}.
+   * @return Ordered list of {@link RegionInfo}.
    * @throws IOException
    */
-  public static List<HRegionInfo> getTableRegions(Connection connection, TableName tableName)
+  public static List<RegionInfo> getTableRegions(Connection connection, TableName tableName)
   throws IOException {
     return getTableRegions(connection, tableName, false);
   }
@@ -461,22 +469,21 @@ public class MetaTableAccessor {
    * @param tableName table we're looking for
    * @param excludeOfflinedSplitParents If true, do not include offlined split
    * parents in the return.
-   * @return Ordered list of {@link HRegionInfo}.
+   * @return Ordered list of {@link RegionInfo}.
    * @throws IOException
    */
-  public static List<HRegionInfo> getTableRegions(Connection connection,
+  public static List<RegionInfo> getTableRegions(Connection connection,
       TableName tableName, final boolean excludeOfflinedSplitParents)
   throws IOException {
-    List<Pair<HRegionInfo, ServerName>> result =
+    List<Pair<RegionInfo, ServerName>> result =
         getTableRegionsAndLocations(connection, tableName, excludeOfflinedSplitParents);
-    return getListOfHRegionInfos(result);
+    return getListOfRegionInfos(result);
   }
 
-  @Nullable
-  static List<HRegionInfo> getListOfHRegionInfos(final List<Pair<HRegionInfo, ServerName>> pairs) {
-    if (pairs == null || pairs.isEmpty()) return null;
-    List<HRegionInfo> result = new ArrayList<>(pairs.size());
-    for (Pair<HRegionInfo, ServerName> pair: pairs) {
+  static List<RegionInfo> getListOfRegionInfos(final List<Pair<RegionInfo, ServerName>> pairs) {
+    if (pairs == null || pairs.isEmpty()) return Collections.EMPTY_LIST;
+    List<RegionInfo> result = new ArrayList<>(pairs.size());
+    for (Pair<RegionInfo, ServerName> pair: pairs) {
       result.add(pair.getFirst());
     }
     return result;
@@ -488,7 +495,7 @@ public class MetaTableAccessor {
    * @return True if <code>current</code> tablename is equal to
    * <code>tableName</code>
    */
-  static boolean isInsideTable(final HRegionInfo current, final TableName tableName) {
+  static boolean isInsideTable(final RegionInfo current, final TableName tableName) {
     return tableName.equals(current.getTable());
   }
 
@@ -587,7 +594,7 @@ public class MetaTableAccessor {
    * @return Return list of regioninfos and server.
    * @throws IOException
    */
-  public static List<Pair<HRegionInfo, ServerName>>
+  public static List<Pair<RegionInfo, ServerName>>
     getTableRegionsAndLocations(Connection connection, TableName tableName)
       throws IOException {
     return getTableRegionsAndLocations(connection, tableName, true);
@@ -601,26 +608,26 @@ public class MetaTableAccessor {
    * @return Return list of regioninfos and server addresses.
    * @throws IOException
    */
-  public static List<Pair<HRegionInfo, ServerName>> getTableRegionsAndLocations(
+  public static List<Pair<RegionInfo, ServerName>> getTableRegionsAndLocations(
       Connection connection, @Nullable final TableName tableName,
       final boolean excludeOfflinedSplitParents) throws IOException {
     if (tableName != null && tableName.equals(TableName.META_TABLE_NAME)) {
       throw new IOException("This method can't be used to locate meta regions;"
         + " use MetaTableLocator instead");
     }
-    // Make a version of CollectingVisitor that collects HRegionInfo and ServerAddress
-    CollectingVisitor<Pair<HRegionInfo, ServerName>> visitor =
-      new CollectingVisitor<Pair<HRegionInfo, ServerName>>() {
+    // Make a version of CollectingVisitor that collects RegionInfo and ServerAddress
+    CollectingVisitor<Pair<RegionInfo, ServerName>> visitor =
+      new CollectingVisitor<Pair<RegionInfo, ServerName>>() {
         private RegionLocations current = null;
 
         @Override
         public boolean visit(Result r) throws IOException {
           current = getRegionLocations(r);
           if (current == null || current.getRegionLocation().getRegionInfo() == null) {
-            LOG.warn("No serialized HRegionInfo in " + r);
+            LOG.warn("No serialized RegionInfo in " + r);
             return true;
           }
-          HRegionInfo hri = current.getRegionLocation().getRegionInfo();
+          RegionInfo hri = current.getRegionLocation().getRegionInfo();
           if (excludeOfflinedSplitParents && hri.isSplitParent()) return true;
           // Else call super and add this Result to the collection.
           return super.visit(r);
@@ -652,10 +659,10 @@ public class MetaTableAccessor {
    * catalog regions).
    * @throws IOException
    */
-  public static NavigableMap<HRegionInfo, Result>
+  public static NavigableMap<RegionInfo, Result>
   getServerUserRegions(Connection connection, final ServerName serverName)
     throws IOException {
-    final NavigableMap<HRegionInfo, Result> hris = new TreeMap<>();
+    final NavigableMap<RegionInfo, Result> hris = new TreeMap<>();
     // Fill the above hris map with entries from hbase:meta that have the passed
     // servername.
     CollectingVisitor<Result> v = new CollectingVisitor<Result>() {
@@ -741,9 +748,9 @@ public class MetaTableAccessor {
       startRow =
           getTableStartRowForMeta(tableName, QueryType.REGION);
       if (row != null) {
-        HRegionInfo closestRi =
+        RegionInfo closestRi =
             getClosestRegionInfo(connection, tableName, row);
-        startRow = HRegionInfo
+        startRow = RegionInfo
             .createRegionName(tableName, closestRi.getStartKey(), HConstants.ZEROES, false);
       }
       stopRow =
@@ -813,11 +820,11 @@ public class MetaTableAccessor {
    * @throws java.io.IOException
    */
   @NonNull
-  public static HRegionInfo getClosestRegionInfo(Connection connection,
+  public static RegionInfo getClosestRegionInfo(Connection connection,
       @NonNull final TableName tableName,
       @NonNull final byte[] row)
       throws IOException {
-    byte[] searchRow = HRegionInfo.createRegionName(tableName, row, HConstants.NINES, false);
+    byte[] searchRow = RegionInfo.createRegionName(tableName, row, HConstants.NINES, false);
     Scan scan = getMetaScan(connection, 1);
     scan.setReversed(true);
     scan.setStartRow(searchRow);
@@ -827,9 +834,9 @@ public class MetaTableAccessor {
         throw new TableNotFoundException("Cannot find row in META " +
             " for table: " + tableName + ", row=" + Bytes.toStringBinary(row));
       }
-      HRegionInfo regionInfo = getHRegionInfo(result);
+      RegionInfo regionInfo = getRegionInfo(result);
       if (regionInfo == null) {
-        throw new IOException("HRegionInfo was null or empty in Meta for " +
+        throw new IOException("RegionInfo was null or empty in Meta for " +
             tableName + ", row=" + Bytes.toStringBinary(row));
       }
       return regionInfo;
@@ -879,7 +886,7 @@ public class MetaTableAccessor {
     return replicaId == 0
       ? HConstants.SERVER_QUALIFIER
       : Bytes.toBytes(HConstants.SERVER_QUALIFIER_STR + META_REPLICA_ID_DELIMITER
-      + String.format(HRegionInfo.REPLICA_ID_FORMAT, replicaId));
+      + String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId));
   }
 
   /**
@@ -892,7 +899,7 @@ public class MetaTableAccessor {
     return replicaId == 0
       ? HConstants.STARTCODE_QUALIFIER
       : Bytes.toBytes(HConstants.STARTCODE_QUALIFIER_STR + META_REPLICA_ID_DELIMITER
-      + String.format(HRegionInfo.REPLICA_ID_FORMAT, replicaId));
+      + String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId));
   }
 
   /**
@@ -905,7 +912,7 @@ public class MetaTableAccessor {
     return replicaId == 0
       ? HConstants.SEQNUM_QUALIFIER
       : Bytes.toBytes(HConstants.SEQNUM_QUALIFIER_STR + META_REPLICA_ID_DELIMITER
-      + String.format(HRegionInfo.REPLICA_ID_FORMAT, replicaId));
+      + String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId));
   }
 
   /**
@@ -975,7 +982,7 @@ public class MetaTableAccessor {
   @Nullable
   public static RegionLocations getRegionLocations(final Result r) {
     if (r == null) return null;
-    HRegionInfo regionInfo = getHRegionInfo(r, getRegionInfoColumn());
+    RegionInfo regionInfo = getRegionInfo(r, getRegionInfoColumn());
     if (regionInfo == null) return null;
 
     List<HRegionLocation> locations = new ArrayList<>(1);
@@ -1021,37 +1028,37 @@ public class MetaTableAccessor {
    * @param replicaId the replicaId for the HRegionLocation
    * @return HRegionLocation parsed from the given meta row Result for the given replicaId
    */
-  private static HRegionLocation getRegionLocation(final Result r, final HRegionInfo regionInfo,
+  private static HRegionLocation getRegionLocation(final Result r, final RegionInfo regionInfo,
                                                    final int replicaId) {
     ServerName serverName = getServerName(r, replicaId);
     long seqNum = getSeqNumDuringOpen(r, replicaId);
-    HRegionInfo replicaInfo = RegionReplicaUtil.getRegionInfoForReplica(regionInfo, replicaId);
+    RegionInfo replicaInfo = RegionReplicaUtil.getRegionInfoForReplica(regionInfo, replicaId);
     return new HRegionLocation(replicaInfo, serverName, seqNum);
   }
 
   /**
-   * Returns HRegionInfo object from the column
+   * Returns RegionInfo object from the column
    * HConstants.CATALOG_FAMILY:HConstants.REGIONINFO_QUALIFIER of the catalog
    * table Result.
    * @param data a Result object from the catalog table scan
-   * @return HRegionInfo or null
+   * @return RegionInfo or null
    */
-  public static HRegionInfo getHRegionInfo(Result data) {
-    return getHRegionInfo(data, HConstants.REGIONINFO_QUALIFIER);
+  public static RegionInfo getRegionInfo(Result data) {
+    return getRegionInfo(data, HConstants.REGIONINFO_QUALIFIER);
   }
 
   /**
-   * Returns the HRegionInfo object from the column {@link HConstants#CATALOG_FAMILY} and
+   * Returns the RegionInfo object from the column {@link HConstants#CATALOG_FAMILY} and
    * <code>qualifier</code> of the catalog table result.
    * @param r a Result object from the catalog table scan
    * @param qualifier Column family qualifier
-   * @return An HRegionInfo instance or null.
+   * @return An RegionInfo instance or null.
    */
   @Nullable
-  private static HRegionInfo getHRegionInfo(final Result r, byte [] qualifier) {
+  private static RegionInfo getRegionInfo(final Result r, byte [] qualifier) {
     Cell cell = r.getColumnLatestCell(getCatalogFamily(), qualifier);
     if (cell == null) return null;
-    return HRegionInfo.parseFromOrNull(cell.getValueArray(),
+    return RegionInfo.parseFromOrNull(cell.getValueArray(),
       cell.getValueOffset(), cell.getValueLength());
   }
 
@@ -1059,12 +1066,12 @@ public class MetaTableAccessor {
    * Returns the daughter regions by reading the corresponding columns of the catalog table
    * Result.
    * @param data a Result object from the catalog table scan
-   * @return a pair of HRegionInfo or PairOfSameType(null, null) if the region is not a split
+   * @return a pair of RegionInfo or PairOfSameType(null, null) if the region is not a split
    * parent
    */
-  public static PairOfSameType<HRegionInfo> getDaughterRegions(Result data) {
-    HRegionInfo splitA = getHRegionInfo(data, HConstants.SPLITA_QUALIFIER);
-    HRegionInfo splitB = getHRegionInfo(data, HConstants.SPLITB_QUALIFIER);
+  public static PairOfSameType<RegionInfo> getDaughterRegions(Result data) {
+    RegionInfo splitA = getRegionInfo(data, HConstants.SPLITA_QUALIFIER);
+    RegionInfo splitB = getRegionInfo(data, HConstants.SPLITB_QUALIFIER);
 
     return new PairOfSameType<>(splitA, splitB);
   }
@@ -1073,12 +1080,12 @@ public class MetaTableAccessor {
    * Returns the merge regions by reading the corresponding columns of the catalog table
    * Result.
    * @param data a Result object from the catalog table scan
-   * @return a pair of HRegionInfo or PairOfSameType(null, null) if the region is not a split
+   * @return a pair of RegionInfo or PairOfSameType(null, null) if the region is not a split
    * parent
    */
-  public static PairOfSameType<HRegionInfo> getMergeRegions(Result data) {
-    HRegionInfo mergeA = getHRegionInfo(data, HConstants.MERGEA_QUALIFIER);
-    HRegionInfo mergeB = getHRegionInfo(data, HConstants.MERGEB_QUALIFIER);
+  public static PairOfSameType<RegionInfo> getMergeRegions(Result data) {
+    RegionInfo mergeA = getRegionInfo(data, HConstants.MERGEA_QUALIFIER);
+    RegionInfo mergeB = getRegionInfo(data, HConstants.MERGEB_QUALIFIER);
 
     return new PairOfSameType<>(mergeA, mergeB);
   }
@@ -1222,7 +1229,7 @@ public class MetaTableAccessor {
 
     @Override
     public boolean visit(Result rowResult) throws IOException {
-      HRegionInfo info = getHRegionInfo(rowResult);
+      RegionInfo info = getRegionInfo(rowResult);
       if (info == null) {
         return true;
       }
@@ -1251,7 +1258,7 @@ public class MetaTableAccessor {
 
     @Override
     public final boolean visit(Result rowResult) throws IOException {
-      HRegionInfo info = getHRegionInfo(rowResult);
+      RegionInfo info = getRegionInfo(rowResult);
       if (info == null) {
         return true;
       }
@@ -1298,7 +1305,7 @@ public class MetaTableAccessor {
   /**
    * Generates and returns a Put containing the region into for the catalog table
    */
-  public static Put makePutFromRegionInfo(HRegionInfo regionInfo)
+  public static Put makePutFromRegionInfo(RegionInfo regionInfo)
     throws IOException {
     return makePutFromRegionInfo(regionInfo, EnvironmentEdgeManager.currentTime());
   }
@@ -1306,7 +1313,7 @@ public class MetaTableAccessor {
   /**
    * Generates and returns a Put containing the region into for the catalog table
    */
-  public static Put makePutFromRegionInfo(HRegionInfo regionInfo, long ts)
+  public static Put makePutFromRegionInfo(RegionInfo regionInfo, long ts)
     throws IOException {
     Put put = new Put(regionInfo.getRegionName(), ts);
     addRegionInfo(put, regionInfo);
@@ -1317,7 +1324,7 @@ public class MetaTableAccessor {
    * Generates and returns a Delete containing the region info for the catalog
    * table
    */
-  public static Delete makeDeleteFromRegionInfo(HRegionInfo regionInfo) {
+  public static Delete makeDeleteFromRegionInfo(RegionInfo regionInfo) {
     long now = EnvironmentEdgeManager.currentTime();
     return makeDeleteFromRegionInfo(regionInfo, now);
   }
@@ -1326,7 +1333,7 @@ public class MetaTableAccessor {
    * Generates and returns a Delete containing the region info for the catalog
    * table
    */
-  public static Delete makeDeleteFromRegionInfo(HRegionInfo regionInfo, long ts) {
+  public static Delete makeDeleteFromRegionInfo(RegionInfo regionInfo, long ts) {
     if (regionInfo == null) {
       throw new IllegalArgumentException("Can't make a delete for null region");
     }
@@ -1356,14 +1363,14 @@ public class MetaTableAccessor {
   /**
    * Adds split daughters to the Put
    */
-  public static Put addDaughtersToPut(Put put, HRegionInfo splitA, HRegionInfo splitB) {
+  public static Put addDaughtersToPut(Put put, RegionInfo splitA, RegionInfo splitB) {
     if (splitA != null) {
       put.addImmutable(
-        HConstants.CATALOG_FAMILY, HConstants.SPLITA_QUALIFIER, splitA.toByteArray());
+        HConstants.CATALOG_FAMILY, HConstants.SPLITA_QUALIFIER, RegionInfo.toByteArray(splitA));
     }
     if (splitB != null) {
       put.addImmutable(
-        HConstants.CATALOG_FAMILY, HConstants.SPLITB_QUALIFIER, splitB.toByteArray());
+        HConstants.CATALOG_FAMILY, HConstants.SPLITB_QUALIFIER, RegionInfo.toByteArray(splitB));
     }
     return put;
   }
@@ -1505,7 +1512,7 @@ public class MetaTableAccessor {
    * @throws IOException if problem connecting or updating meta
    */
   public static void addRegionToMeta(Connection connection,
-                                     HRegionInfo regionInfo)
+                                     RegionInfo regionInfo)
     throws IOException {
     putToMetaTable(connection, makePutFromRegionInfo(regionInfo));
     LOG.info("Added " + regionInfo.getRegionNameAsString());
@@ -1518,7 +1525,7 @@ public class MetaTableAccessor {
    * @param regionInfo region information
    * @throws IOException if problem connecting or updating meta
    */
-  public static void addRegionToMeta(Table meta, HRegionInfo regionInfo) throws IOException {
+  public static void addRegionToMeta(Table meta, RegionInfo regionInfo) throws IOException {
     addRegionToMeta(meta, regionInfo, null, null);
   }
 
@@ -1526,7 +1533,7 @@ public class MetaTableAccessor {
    * Adds a (single) hbase:meta row for the specified new region and its daughters. Note that this
    * does not add its daughter's as different rows, but adds information about the daughters
    * in the same row as the parent. Use
-   * {@link #splitRegion(Connection, HRegionInfo, HRegionInfo, HRegionInfo, ServerName,int,boolean)}
+   * {@link #splitRegion(Connection, RegionInfo, RegionInfo, RegionInfo, ServerName,int,boolean)}
    * if you want to do that.
    * @param meta the Table for META
    * @param regionInfo region information
@@ -1534,8 +1541,8 @@ public class MetaTableAccessor {
    * @param splitB second split daughter of the parent regionInfo
    * @throws IOException if problem connecting or updating meta
    */
-  public static void addRegionToMeta(Table meta, HRegionInfo regionInfo,
-                                     HRegionInfo splitA, HRegionInfo splitB) throws IOException {
+  public static void addRegionToMeta(Table meta, RegionInfo regionInfo,
+                                     RegionInfo splitA, RegionInfo splitB) throws IOException {
     Put put = makePutFromRegionInfo(regionInfo);
     addDaughtersToPut(put, splitA, splitB);
     meta.put(put);
@@ -1551,7 +1558,7 @@ public class MetaTableAccessor {
    * Adds a (single) hbase:meta row for the specified new region and its daughters. Note that this
    * does not add its daughter's as different rows, but adds information about the daughters
    * in the same row as the parent. Use
-   * {@link #splitRegion(Connection, HRegionInfo, HRegionInfo, HRegionInfo, ServerName,int,boolean)}
+   * {@link #splitRegion(Connection, RegionInfo, RegionInfo, RegionInfo, ServerName,int,boolean)}
    * if you want to do that.
    * @param connection connection we're using
    * @param regionInfo region information
@@ -1559,8 +1566,8 @@ public class MetaTableAccessor {
    * @param splitB second split daughter of the parent regionInfo
    * @throws IOException if problem connecting or updating meta
    */
-  public static void addRegionToMeta(Connection connection, HRegionInfo regionInfo,
-                                     HRegionInfo splitA, HRegionInfo splitB) throws IOException {
+  public static void addRegionToMeta(Connection connection, RegionInfo regionInfo,
+                                     RegionInfo splitA, RegionInfo splitB) throws IOException {
     Table meta = getMetaHTable(connection);
     try {
       addRegionToMeta(meta, regionInfo, splitA, splitB);
@@ -1576,7 +1583,7 @@ public class MetaTableAccessor {
    * @throws IOException if problem connecting or updating meta
    */
   public static void addRegionsToMeta(Connection connection,
-                                      List<HRegionInfo> regionInfos, int regionReplication)
+                                      List<RegionInfo> regionInfos, int regionReplication)
     throws IOException {
     addRegionsToMeta(connection, regionInfos, regionReplication, HConstants.LATEST_TIMESTAMP);
   }
@@ -1589,10 +1596,10 @@ public class MetaTableAccessor {
    * @throws IOException if problem connecting or updating meta
    */
   public static void addRegionsToMeta(Connection connection,
-      List<HRegionInfo> regionInfos, int regionReplication, long ts)
+      List<RegionInfo> regionInfos, int regionReplication, long ts)
           throws IOException {
     List<Put> puts = new ArrayList<>();
-    for (HRegionInfo regionInfo : regionInfos) {
+    for (RegionInfo regionInfo : regionInfos) {
       if (RegionReplicaUtil.isDefaultReplica(regionInfo)) {
         Put put = makePutFromRegionInfo(regionInfo, ts);
         // Add empty locations for region replicas so that number of replicas can be cached
@@ -1614,7 +1621,7 @@ public class MetaTableAccessor {
    * @param openSeqNum the latest sequence number obtained when the region was open
    */
   public static void addDaughter(final Connection connection,
-      final HRegionInfo regionInfo, final ServerName sn, final long openSeqNum)
+      final RegionInfo regionInfo, final ServerName sn, final long openSeqNum)
       throws NotAllMetaRegionsOnlineException, IOException {
     long now = EnvironmentEdgeManager.currentTime();
     Put put = new Put(regionInfo.getRegionName(), now);
@@ -1640,23 +1647,21 @@ public class MetaTableAccessor {
    * @param saveBarrier true if need save replication barrier in meta, used for serial replication
    * @throws IOException
    */
-  public static void mergeRegions(final Connection connection, HRegionInfo mergedRegion,
-      HRegionInfo regionA, HRegionInfo regionB, ServerName sn, int regionReplication,
+  public static void mergeRegions(final Connection connection, RegionInfo mergedRegion,
+      RegionInfo regionA, RegionInfo regionB, ServerName sn, int regionReplication,
       long masterSystemTime, boolean saveBarrier)
           throws IOException {
     Table meta = getMetaHTable(connection);
     try {
-      HRegionInfo copyOfMerged = new HRegionInfo(mergedRegion);
-
       // use the maximum of what master passed us vs local time.
       long time = Math.max(EnvironmentEdgeManager.currentTime(), masterSystemTime);
 
       // Put for parent
-      Put putOfMerged = makePutFromRegionInfo(copyOfMerged, time);
+      Put putOfMerged = makePutFromRegionInfo(mergedRegion, time);
       putOfMerged.addImmutable(HConstants.CATALOG_FAMILY, HConstants.MERGEA_QUALIFIER,
-        regionA.toByteArray());
+        RegionInfo.toByteArray(regionA));
       putOfMerged.addImmutable(HConstants.CATALOG_FAMILY, HConstants.MERGEB_QUALIFIER,
-        regionB.toByteArray());
+          RegionInfo.toByteArray(regionB));
 
       // Deletes for merging regions
       Delete deleteA = makeDeleteFromRegionInfo(regionA, time);
@@ -1707,17 +1712,15 @@ public class MetaTableAccessor {
    * @param sn the location of the region
    * @param saveBarrier true if need save replication barrier in meta, used for serial replication
    */
-  public static void splitRegion(final Connection connection, HRegionInfo parent,
-      HRegionInfo splitA, HRegionInfo splitB, ServerName sn, int regionReplication,
+  public static void splitRegion(final Connection connection, RegionInfo parent,
+      RegionInfo splitA, RegionInfo splitB, ServerName sn, int regionReplication,
       boolean saveBarrier) throws IOException {
     Table meta = getMetaHTable(connection);
     try {
-      HRegionInfo copyOfParent = new HRegionInfo(parent);
-      copyOfParent.setOffline(true);
-      copyOfParent.setSplit(true);
-
       //Put for parent
-      Put putParent = makePutFromRegionInfo(copyOfParent);
+      Put putParent = makePutFromRegionInfo(RegionInfoBuilder.newBuilder(parent)
+                        .setOffline(true)
+                        .setSplit(true).build());
       addDaughtersToPut(putParent, splitA, splitB);
 
       //Puts for daughters
@@ -1878,7 +1881,7 @@ public class MetaTableAccessor {
    * @throws IOException
    */
   public static void updateRegionLocation(Connection connection,
-                                          HRegionInfo regionInfo, ServerName sn, long openSeqNum,
+                                          RegionInfo regionInfo, ServerName sn, long openSeqNum,
                                           long masterSystemTime)
     throws IOException {
     updateLocation(connection, regionInfo, sn, openSeqNum, masterSystemTime);
@@ -1920,7 +1923,7 @@ public class MetaTableAccessor {
    * if the server is down on other end.
    */
   private static void updateLocation(final Connection connection,
-                                     HRegionInfo regionInfo, ServerName sn, long openSeqNum,
+                                     RegionInfo regionInfo, ServerName sn, long openSeqNum,
                                      long masterSystemTime)
     throws IOException {
 
@@ -1943,7 +1946,7 @@ public class MetaTableAccessor {
    * @throws IOException
    */
   public static void deleteRegion(Connection connection,
-                                  HRegionInfo regionInfo)
+                                  RegionInfo regionInfo)
     throws IOException {
     long time = EnvironmentEdgeManager.currentTime();
     Delete delete = new Delete(regionInfo.getRegionName());
@@ -1959,7 +1962,7 @@ public class MetaTableAccessor {
    * @throws IOException
    */
   public static void deleteRegions(Connection connection,
-                                   List<HRegionInfo> regionsInfo) throws IOException {
+                                   List<RegionInfo> regionsInfo) throws IOException {
     deleteRegions(connection, regionsInfo, EnvironmentEdgeManager.currentTime());
   }
   /**
@@ -1968,10 +1971,10 @@ public class MetaTableAccessor {
    * @param regionsInfo list of regions to be deleted from META
    * @throws IOException
    */
-  public static void deleteRegions(Connection connection, List<HRegionInfo> regionsInfo, long ts)
+  public static void deleteRegions(Connection connection, List<RegionInfo> regionsInfo, long ts)
   throws IOException {
     List<Delete> deletes = new ArrayList<>(regionsInfo.size());
-    for (HRegionInfo hri: regionsInfo) {
+    for (RegionInfo hri: regionsInfo) {
       Delete e = new Delete(hri.getRegionName());
       e.addFamily(getCatalogFamily(), ts);
       deletes.add(e);
@@ -1991,26 +1994,26 @@ public class MetaTableAccessor {
    * @throws IOException
    */
   public static void mutateRegions(Connection connection,
-                                   final List<HRegionInfo> regionsToRemove,
-                                   final List<HRegionInfo> regionsToAdd)
+                                   final List<RegionInfo> regionsToRemove,
+                                   final List<RegionInfo> regionsToAdd)
     throws IOException {
     List<Mutation> mutation = new ArrayList<>();
     if (regionsToRemove != null) {
-      for (HRegionInfo hri: regionsToRemove) {
+      for (RegionInfo hri: regionsToRemove) {
         mutation.add(makeDeleteFromRegionInfo(hri));
       }
     }
     if (regionsToAdd != null) {
-      for (HRegionInfo hri: regionsToAdd) {
+      for (RegionInfo hri: regionsToAdd) {
         mutation.add(makePutFromRegionInfo(hri));
       }
     }
     mutateMetaTable(connection, mutation);
     if (regionsToRemove != null && regionsToRemove.size() > 0) {
-      LOG.debug("Deleted " + HRegionInfo.getShortNameToLog(regionsToRemove));
+      LOG.debug("Deleted " + RegionInfo.getShortNameToLog(regionsToRemove));
     }
     if (regionsToAdd != null && regionsToAdd.size() > 0) {
-      LOG.debug("Added " + HRegionInfo.getShortNameToLog(regionsToAdd));
+      LOG.debug("Added " + RegionInfo.getShortNameToLog(regionsToAdd));
     }
   }
 
@@ -2021,7 +2024,7 @@ public class MetaTableAccessor {
    * @throws IOException
    */
   public static void overwriteRegions(Connection connection,
-      List<HRegionInfo> regionInfos, int regionReplication) throws IOException {
+      List<RegionInfo> regionInfos, int regionReplication) throws IOException {
     // use master time for delete marker and the Put
     long now = EnvironmentEdgeManager.currentTime();
     deleteRegions(connection, regionInfos, now);
@@ -2045,7 +2048,7 @@ public class MetaTableAccessor {
    * @throws IOException
    */
   public static void deleteMergeQualifiers(Connection connection,
-                                           final HRegionInfo mergedRegion) throws IOException {
+                                           final RegionInfo mergedRegion) throws IOException {
     long time = EnvironmentEdgeManager.currentTime();
     Delete delete = new Delete(mergedRegion.getRegionName());
     delete.addColumns(getCatalogFamily(), HConstants.MERGEA_QUALIFIER, time);
@@ -2057,10 +2060,10 @@ public class MetaTableAccessor {
       + Bytes.toStringBinary(HConstants.MERGEB_QUALIFIER));
   }
 
-  public static Put addRegionInfo(final Put p, final HRegionInfo hri)
+  public static Put addRegionInfo(final Put p, final RegionInfo hri)
     throws IOException {
     p.addImmutable(getCatalogFamily(), HConstants.REGIONINFO_QUALIFIER,
-      hri.toByteArray());
+        RegionInfo.toByteArray(hri));
     return p;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionLocations.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionLocations.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionLocations.java
index bbff8ce..8b3fbb4 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionLocations.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionLocations.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hbase;
 
 import java.util.Collection;
 
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -143,14 +144,14 @@ public class RegionLocations {
    */
   public RegionLocations remove(HRegionLocation location) {
     if (location == null) return this;
-    if (location.getRegionInfo() == null) return this;
+    if (location.getRegion() == null) return this;
     int replicaId = location.getRegionInfo().getReplicaId();
     if (replicaId >= locations.length) return this;
 
     // check whether something to remove. HRL.compareTo() compares ONLY the
     // serverName. We want to compare the HRI's as well.
     if (locations[replicaId] == null
-        || !location.getRegionInfo().equals(locations[replicaId].getRegionInfo())
+        || RegionInfo.COMPARATOR.compare(location.getRegion(), locations[replicaId].getRegion()) != 0
         || !location.equals(locations[replicaId])) {
       return this;
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
index b2c6969..e539c6a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
@@ -17,9 +17,9 @@
  */
 package org.apache.hadoop.hbase.client;
 
-import java.util.List;
 import java.util.Collection;
 import java.util.EnumSet;
+import java.util.List;
 import java.util.Map;
 import java.util.Optional;
 import java.util.Set;
@@ -29,20 +29,19 @@ import java.util.regex.Pattern;
 
 import org.apache.hadoop.hbase.ClusterStatus;
 import org.apache.hadoop.hbase.ClusterStatus.Option;
-import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.RegionLoad;
 import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.quotas.QuotaFilter;
-import org.apache.hadoop.hbase.quotas.QuotaSettings;
 import org.apache.hadoop.hbase.client.RawAsyncTable.CoprocessorCallable;
 import org.apache.hadoop.hbase.client.replication.TableCFs;
 import org.apache.hadoop.hbase.client.security.SecurityCapability;
+import org.apache.hadoop.hbase.quotas.QuotaFilter;
+import org.apache.hadoop.hbase.quotas.QuotaSettings;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.util.Pair;
+import org.apache.yetus.audience.InterfaceAudience;
 
 import com.google.protobuf.RpcChannel;
 
@@ -306,12 +305,12 @@ public interface AsyncAdmin {
   /**
    * Get all the online regions on a region server.
    */
-  CompletableFuture<List<HRegionInfo>> getOnlineRegions(ServerName serverName);
+  CompletableFuture<List<RegionInfo>> getOnlineRegions(ServerName serverName);
 
   /**
    * Get the regions of a given table.
    */
-  CompletableFuture<List<HRegionInfo>> getTableRegions(TableName tableName);
+  CompletableFuture<List<RegionInfo>> getTableRegions(TableName tableName);
 
   /**
    * Flush a table.

http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
index a3cc89f..3cb8360 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
@@ -32,12 +32,10 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.ClusterStatus;
 import org.apache.hadoop.hbase.ClusterStatus.Option;
-import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.RegionLoad;
 import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.client.RawAsyncTable.CoprocessorCallable;
 import org.apache.hadoop.hbase.client.replication.TableCFs;
 import org.apache.hadoop.hbase.client.security.SecurityCapability;
@@ -46,6 +44,7 @@ import org.apache.hadoop.hbase.quotas.QuotaSettings;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.util.Pair;
+import org.apache.yetus.audience.InterfaceAudience;
 
 import com.google.protobuf.RpcChannel;
 
@@ -215,12 +214,12 @@ public class AsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<List<HRegionInfo>> getOnlineRegions(ServerName serverName) {
+  public CompletableFuture<List<RegionInfo>> getOnlineRegions(ServerName serverName) {
     return wrap(rawAdmin.getOnlineRegions(serverName));
   }
 
   @Override
-  public CompletableFuture<List<HRegionInfo>> getTableRegions(TableName tableName) {
+  public CompletableFuture<List<RegionInfo>> getTableRegions(TableName tableName) {
     return wrap(rawAdmin.getTableRegions(tableName));
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
index 860eef3..9d888e6 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
@@ -57,7 +57,6 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotEnabledException;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicy;
 import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicyFactory;
 import org.apache.hadoop.hbase.exceptions.ClientExceptionsUtil;
@@ -67,6 +66,20 @@ import org.apache.hadoop.hbase.ipc.RpcClientFactory;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
 import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.ExceptionUtil;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.ReflectionUtils;
+import org.apache.hadoop.hbase.util.Threads;
+import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.yetus.audience.InterfaceAudience;
+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.BlockingRpcChannel;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
@@ -110,19 +123,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.Remov
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerResponse;
 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.util.Bytes;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.ExceptionUtil;
-import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.hbase.util.ReflectionUtils;
-import org.apache.hadoop.hbase.util.Threads;
-import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
-import org.apache.hadoop.ipc.RemoteException;
-import org.apache.zookeeper.KeeperException;
-
-import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
 
 import edu.umd.cs.findbugs.annotations.Nullable;
 
@@ -600,13 +600,13 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
         LOG.debug("Table " + tableName + " not enabled");
         return false;
       }
-      List<Pair<HRegionInfo, ServerName>> locations =
+      List<Pair<RegionInfo, ServerName>> locations =
         MetaTableAccessor.getTableRegionsAndLocations(this, tableName, true);
 
       int notDeployed = 0;
       int regionCount = 0;
-      for (Pair<HRegionInfo, ServerName> pair : locations) {
-        HRegionInfo info = pair.getFirst();
+      for (Pair<RegionInfo, ServerName> pair : locations) {
+        RegionInfo info = pair.getFirst();
         if (pair.getSecond() == null) {
           if (LOG.isDebugEnabled()) {
             LOG.debug("Table " + tableName + " has not deployed region " + pair.getFirst()
@@ -675,10 +675,10 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
   @Override
   public List<HRegionLocation> locateRegions(final TableName tableName,
       final boolean useCache, final boolean offlined) throws IOException {
-    List<HRegionInfo> regions = MetaTableAccessor
+    List<RegionInfo> regions = MetaTableAccessor
         .getTableRegions(this, tableName, !offlined);
     final List<HRegionLocation> locations = new ArrayList<>();
-    for (HRegionInfo regionInfo : regions) {
+    for (RegionInfo regionInfo : regions) {
       RegionLocations list = locateRegion(tableName, regionInfo.getStartKey(), useCache, true);
       if (list != null) {
         for (HRegionLocation loc : list.getRegionLocations()) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a11a35a1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FlushRegionCallable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FlushRegionCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FlushRegionCallable.java
index af5c4d6..442bf1d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FlushRegionCallable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FlushRegionCallable.java
@@ -22,16 +22,16 @@ import java.io.IOException;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.ipc.HBaseRpcController;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.yetus.audience.InterfaceAudience;
+
 import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 
 /**
  * A Callable for flushRegion() RPC.
@@ -52,7 +52,7 @@ public class FlushRegionCallable extends RegionAdminServiceCallable<FlushRegionR
   }
 
   public FlushRegionCallable(ClusterConnection connection,
-      RpcControllerFactory rpcControllerFactory, HRegionInfo regionInfo,
+      RpcControllerFactory rpcControllerFactory, RegionInfo regionInfo,
       boolean writeFlushWalMarker) {
     this(connection, rpcControllerFactory, regionInfo.getTable(), regionInfo.getRegionName(),
       regionInfo.getStartKey(), writeFlushWalMarker);