You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by op...@apache.org on 2019/03/28 09:56:33 UTC

[hbase] branch HBASE-21879 updated (2e7cc5c -> 109c3ca)

This is an automated email from the ASF dual-hosted git repository.

openinx pushed a change to branch HBASE-21879
in repository https://gitbox.apache.org/repos/asf/hbase.git.


    from 2e7cc5c  HBASE-22025 RAT check fails in nightlies; fails on (old) test data files.
     new 0da5b26  HBASE-21810 bulkload support set hfile compression on client
     new eb44482  HBASE-21977 Skip replay WAL and update seqid when open regions restored from snapshot
     new 2a9eab9  HBASE-22045 Mutable range histogram reports incorrect outliers
     new 0fb208f  HBASE-22044 ByteBufferUtils should not be IA.Public API
     new 26a6761  HBASE-22022 nightly fails rat check down in the dev-support/hbase_nightly_source-artifact.sh check
     new 05abf94  HBASE-22002 Remove the deprecated methods in Admin interface
     new c37b961  HBASE-20662 Increasing space quota on a violated table does not remove SpaceViolationPolicy.DISABLE enforcement
     new e5dd8d2  HBASE-22032 KeyValue validation should check for null byte array
     new 44d4723  HBASE-22039 Should add the synchronous parameter for the XXXSwitch method in AsyncAdmin
     new 2e3f1f0  HBASE-21926 Profiler servlet
     new a67d08b  HBASE-22040 Add mergeRegionsAsync with a List of region names method in AsyncAdmin
     new 3c8715a  HBASE-22009 Improve RSGroupInfoManagerImpl#getDefaultServers()
     new 22f9fad  HBASE-21963 Add a script for building and verifying release candidate
     new 2a92915  HBASE-22056 Unexpected blank line in ClusterConnection
     new c43b7c8  HBASE-22015 UserPermission should be annotated as InterfaceAudience.Public
     new 9063b90  HBASE-22042 Missing @Override annotation for RawAsyncTableImpl.scan
     new d637e30  HBASE-22061 SplitTableRegionProcedure should hold the lock of its daughter regions
     new 9617279  HBASE-22077 Expose sleep time as CLI argument of IntergationTestBackupRestore (Tamas Adami)
     new 4311e07  HBASE-22063 Deprecated Admin.deleteSnapshot(byte[])
     new 1515fe1  HBASE-22065 Add listTableDescriptors(List<TableName>) method in AsyncAdmin
     new dac7e80  HBASE-22051 Expect values are hard-coded in the verifications of TestRSGroupsBasics
     new 8204286  HBASE-21991: Fix MetaMetrics issues - [Race condition, Faulty remove logic], few improvements
     new 9cde2e9  HBASE-21965 Fix failed split and merge transactions that have failed to roll back
     new d200634  HBASE-22064 Remove Admin.deleteSnapshot(byte[])
     new 2e57be1  HBASE-22095 Taking a snapshot fails in local mode
     new f337f42  HBASE-21895 - Error prone upgrade
     new 2169a37  Revert "HBASE-21965 Fix failed split and merge transactions that have failed to roll back"
     new 90a5db4  HBASE-22093 Combine TestRestoreSnapshotFromClientWithRegionReplicas to CloneSnapshotFromClientAfterSplittingRegionTestBase#testCloneSnapshotAfterSplittingRegion
     new 227ca4b  HBASE-22052 pom cleaning; filter out jersey-core in hadoop2 to match hadoop3 and remove redunant version specifications
     new 69d2615  Add 2.0.5 to the downloads page in place of 2.0.4.
     new 33f3444  HBASE-21619. Fix warning message caused by incorrect ternary operator evaluation
     new e694a4a  HBASE-21964 unset Quota by Throttle Type
     new ae0b6b6  HBASE-22074 Should use procedure store to persist the state in reportRegionStateTransition
     new 8fc81c5  Add 2.1.4 to the downloads page in place of 2.1.3
     new 6ec0c6b     Add 2.1.4 to the downloads page in place of 2.1.3; ADDENDUM
     new 116a62d  Fix the download links for tgzs for 2.0.5
     new 975b2aa  HBASE-22067 Fix log in StochasticLoadBalancer when balancer is ill-fit for cluster size
     new a3edb18  Add a 'notices' column to downloads where we can call out issues
     new 81ee1db  HBASE-21911 Move getUserPermissions from regionserver to master
     new 39f9794  HBASE-22102 Remove AsyncAdmin.isTableAvailable(TableName, byte[][])
     new cc85b6d  HBASE-22094: Throw TableNotFoundException if table not exists in AsyncAdmin.compact
     new 03db5fe  HBASE-22097 Modify the description of split command in shell
     new def29c2  HBASE-22053 Changed zookeeper URL from http://hadoop.apache.org to https://zookeeper.apache.org Replaced version 3.1.1 with 3.3.3 as 3.1.1 is deprecated.
     new ecf0826  HBASE-22082 Should not use an instance to access static members, which will increases compilation costs.
     new a5caeb5  HBASE-22092 Typo in block cache monitoring documentation
     new c2bee58  HBASE-22092 Typo in block cache monitoring documentation
     new 726d2d5  HBASE-21918 the use cases in doc should use Cell instead of KeyValue
     new a353a88  HBASE-22057 Cap the size of the nodes we delete in one ZK.multi call
     new 109c3ca  HBASE-22100 False positive for error prone warnings in pre commit job

The 49 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 bin/hbase                                          |    1 +
 dev-support/hbase-personality.sh                   |   18 +
 dev-support/hbase-vote.sh                          |  183 ++++
 dev-support/hbase_nightly_source-artifact.sh       |    5 +-
 .../archetypes/exemplars/client/HelloHBase.java    |    5 +-
 .../exemplars/shaded_client/HelloHBase.java        |    5 +-
 .../hadoop/hbase/backup/impl/BackupManager.java    |   12 +-
 .../hadoop/hbase/backup/TestIncrementalBackup.java |    5 +-
 hbase-build-configuration/pom.xml                  |   65 +-
 hbase-build-support/hbase-error-prone/pom.xml      |   86 --
 .../hadoop/hbase/errorprone/AlwaysPasses.java      |   40 -
 hbase-build-support/pom.xml                        |   78 --
 .../java/org/apache/hadoop/hbase/client/Admin.java | 1025 ++------------------
 .../org/apache/hadoop/hbase/client/AsyncAdmin.java |  130 ++-
 .../hadoop/hbase/client/AsyncHBaseAdmin.java       |   44 +-
 .../hadoop/hbase/client/ClusterConnection.java     |    2 -
 .../hbase/client/ConnectionImplementation.java     |    8 +
 .../org/apache/hadoop/hbase/client/HBaseAdmin.java |  569 ++---------
 .../org/apache/hadoop/hbase/client/HTable.java     |   66 +-
 .../hadoop/hbase/client/RawAsyncHBaseAdmin.java    |  172 ++--
 .../hadoop/hbase/client/RawAsyncTableImpl.java     |    1 +
 .../hbase/client/ShortCircuitMasterConnection.java |    8 +
 .../hadoop/hbase/quotas/QuotaSettingsFactory.java  |   74 ++
 .../apache/hadoop/hbase/quotas/QuotaTableUtil.java |   36 +-
 .../hbase/security/access/AccessControlClient.java |   92 +-
 .../hbase/security/access/AccessControlUtil.java   |   59 +-
 .../security/access/GetUserPermissionsRequest.java |  136 +++
 .../hbase/security/access/GlobalPermission.java    |   17 +-
 .../hbase/security/access/NamespacePermission.java |   21 +-
 .../hadoop/hbase/security/access/Permission.java   |   32 +-
 .../security/access/ShadedAccessControlUtil.java   |   69 +-
 .../hbase/security/access/TablePermission.java     |   67 +-
 .../hbase/security/access/UserPermission.java      |   85 +-
 .../hadoop/hbase/shaded/protobuf/ProtobufUtil.java |   32 +-
 .../hbase/shaded/protobuf/RequestConverter.java    |   28 +-
 .../java/org/apache/hadoop/hbase/KeyValueUtil.java |    8 +-
 .../apache/hadoop/hbase/util/ByteBufferUtils.java  |    2 +-
 .../org/apache/hadoop/hbase/util/FutureUtils.java  |    2 +-
 hbase-common/src/main/resources/hbase-default.xml  |    6 +-
 .../java/org/apache/hadoop/hbase/TestKeyValue.java |   16 +-
 hbase-endpoint/pom.xml                             |    3 -
 .../coprocessor/TestCoprocessorTableEndpoint.java  |    2 +-
 .../hadoop/metrics2/lib/MutableRangeHistogram.java |    2 +-
 hbase-http/pom.xml                                 |    3 -
 .../org/apache/hadoop/hbase/http/HttpServer.java   |   24 +-
 ...rizedServlet.java => ProfileOutputServlet.java} |   33 +-
 .../apache/hadoop/hbase/http/ProfileServlet.java   |  373 +++++++
 .../org/apache/hadoop/hbase/util/ProcessUtils.java |   70 ++
 hbase-it/pom.xml                                   |   24 +-
 .../hadoop/hbase/DistributedHBaseCluster.java      |    6 +-
 .../hadoop/hbase/IntegrationTestBackupRestore.java |   24 +-
 .../hbase/IntegrationTestDDLMasterFailover.java    |   13 +-
 .../hadoop/hbase/IntegrationTestIngestWithMOB.java |    2 +-
 .../hbase/IntegrationTestRegionReplicaPerf.java    |    2 +-
 .../apache/hadoop/hbase/chaos/actions/Action.java  |    2 +-
 .../actions/CompactRandomRegionOfTableAction.java  |    9 +-
 .../actions/FlushRandomRegionOfTableAction.java    |    9 +-
 .../MergeRandomAdjacentRegionsOfTableAction.java   |    9 +-
 .../chaos/actions/MoveRegionsOfTableAction.java    |    6 +-
 .../actions/SplitRandomRegionOfTableAction.java    |   11 +-
 .../hbase/mapreduce/IntegrationTestBulkLoad.java   |    4 +-
 .../hadoop/hbase/mttr/IntegrationTestMTTR.java     |    7 +-
 .../hbase/rsgroup/IntegrationTestRSGroup.java      |    2 +-
 hbase-mapreduce/pom.xml                            |    3 -
 .../hadoop/hbase/mapreduce/HFileOutputFormat2.java |   15 +-
 .../apache/hadoop/hbase/PerformanceEvaluation.java |    2 +-
 .../hbase/mapreduce/TestHFileOutputFormat2.java    |   57 +-
 .../TestMultiTableSnapshotInputFormatImpl.java     |    5 +
 .../hadoop/hbase/snapshot/TestExportSnapshot.java  |   34 +-
 .../snapshot/TestExportSnapshotNoCluster.java      |    3 +-
 .../hadoop/hbase/procedure2/util/DelayedUtil.java  |    5 +
 hbase-protocol-shaded/pom.xml                      |    2 -
 .../src/main/protobuf/AccessControl.proto          |    3 +
 .../src/main/protobuf/Admin.proto                  |    3 +-
 .../src/main/protobuf/Master.proto                 |    2 +
 .../src/main/protobuf/MasterProcedure.proto        |   11 +
 .../src/main/protobuf/RegionServerStatus.proto     |    1 +
 hbase-rest/pom.xml                                 |   28 +-
 .../hbase/rest/NamespacesInstanceResource.java     |   19 +-
 .../apache/hadoop/hbase/rest/SchemaResource.java   |    4 +-
 .../src/main/resources/hbase-webapps/rest/rest.jsp |    1 +
 .../hadoop/hbase/rest/PerformanceEvaluation.java   |    4 +-
 .../hbase/rsgroup/RSGroupInfoManagerImpl.java      |   18 +-
 .../hadoop/hbase/rsgroup/TestRSGroupsBasics.java   |   58 +-
 hbase-server/pom.xml                               |   26 +
 .../hbase/tmpl/master/MasterStatusTmpl.jamon       |    1 +
 .../hbase/tmpl/regionserver/RSStatusTmpl.jamon     |    1 +
 .../hbase/client/ClientSideRegionScanner.java      |    1 +
 .../hadoop/hbase/coprocessor/BulkLoadObserver.java |    4 +-
 .../hadoop/hbase/coprocessor/MasterObserver.java   |   30 +
 .../hadoop/hbase/coprocessor/MetaTableMetrics.java |  118 ++-
 .../apache/hadoop/hbase/master/CatalogJanitor.java |    6 +-
 .../org/apache/hadoop/hbase/master/HMaster.java    |   16 +-
 .../hadoop/hbase/master/MasterCoprocessorHost.java |   20 +
 .../hadoop/hbase/master/MasterRpcServices.java     |   62 +-
 .../hbase/master/assignment/AssignProcedure.java   |    6 +-
 .../hbase/master/assignment/AssignmentManager.java |   12 +-
 .../master/assignment/CloseRegionProcedure.java    |   26 +-
 .../master/assignment/OpenRegionProcedure.java     |   60 +-
 .../assignment/RegionRemoteProcedureBase.java      |  219 ++++-
 .../hbase/master/assignment/RegionStateStore.java  |    8 +-
 .../assignment/SplitTableRegionProcedure.java      |   20 +
 .../assignment/TransitRegionStateProcedure.java    |  208 +---
 .../hbase/master/assignment/UnassignProcedure.java |    4 +-
 .../master/balancer/RegionLocationFinder.java      |    4 +-
 .../master/balancer/StochasticLoadBalancer.java    |    2 +-
 .../master/normalizer/SplitNormalizationPlan.java  |    6 +-
 .../master/procedure/RSProcedureDispatcher.java    |   54 +-
 .../hbase/master/snapshot/TakeSnapshotHandler.java |    2 +-
 .../hadoop/hbase/mob/ExpiredMobFileCleaner.java    |    6 +-
 .../hbase/quotas/GlobalQuotaSettingsImpl.java      |  122 ++-
 .../hadoop/hbase/quotas/MasterQuotaManager.java    |   10 +
 .../hadoop/hbase/quotas/QuotaObserverChore.java    |   53 +-
 .../org/apache/hadoop/hbase/quotas/QuotaUtil.java  |   34 +
 .../hbase/quotas/SpaceQuotaRefresherChore.java     |   16 +-
 .../DisableTableViolationPolicyEnforcement.java    |   38 +-
 .../hadoop/hbase/regionserver/CompactSplit.java    |    4 +-
 .../DelimitedKeyPrefixRegionSplitPolicy.java       |    4 +-
 .../apache/hadoop/hbase/regionserver/HRegion.java  |   32 +-
 .../hadoop/hbase/regionserver/HRegionServer.java   |   10 +-
 .../hadoop/hbase/regionserver/RSRpcServices.java   |    8 +-
 .../hbase/regionserver/RegionServerServices.java   |   31 +-
 .../hadoop/hbase/regionserver/SplitRequest.java    |    5 +-
 .../regionserver/handler/AssignRegionHandler.java  |   14 +-
 .../regionserver/handler/CloseRegionHandler.java   |    3 +-
 .../regionserver/handler/OpenRegionHandler.java    |   14 +-
 .../handler/UnassignRegionHandler.java             |   15 +-
 .../hbase/security/access/AccessControlLists.java  |   22 +-
 .../hbase/security/access/AccessController.java    |  113 +--
 .../hbase/snapshot/RestoreSnapshotHelper.java      |    6 +-
 .../hbase/snapshot/SnapshotDescriptionUtils.java   |    6 +-
 .../hadoop/hbase/snapshot/SnapshotManifest.java    |    2 +-
 .../java/org/apache/hadoop/hbase/tool/Canary.java  |   10 +-
 .../hadoop/hbase/tool/LoadIncrementalHFiles.java   |    7 +-
 .../org/apache/hadoop/hbase/util/HBaseFsck.java    |   12 +-
 .../apache/hadoop/hbase/util/LossyCounting.java    |   11 +-
 .../main/resources/hbase-webapps/master/header.jsp |    1 +
 .../resources/hbase-webapps/master/rsgroup.jsp     |    6 +-
 .../main/resources/hbase-webapps/master/table.jsp  |    2 +-
 .../hbase-webapps/regionserver/header.jsp          |    1 +
 .../apache/hadoop/hbase/HBaseTestingUtility.java   |   29 +-
 .../hadoop/hbase/TestClientClusterStatus.java      |   24 +-
 .../org/apache/hadoop/hbase/TestNamespace.java     |   36 +-
 .../hbase/TestPartialResultsFromClientSide.java    |    2 +-
 .../org/apache/hadoop/hbase/TestRegionLoad.java    |   17 +-
 ...shotFromClientAfterSplittingRegionTestBase.java |    2 +
 .../CloneSnapshotFromClientNormalTestBase.java     |    2 +-
 .../client/CloneSnapshotFromClientTestBase.java    |   16 +-
 .../RestoreSnapshotFromClientSimpleTestBase.java   |    3 +-
 .../client/RestoreSnapshotFromClientTestBase.java  |   16 +-
 .../org/apache/hadoop/hbase/client/TestAdmin1.java |  338 ++++---
 .../org/apache/hadoop/hbase/client/TestAdmin2.java |  140 ++-
 .../hbase/client/TestAlwaysSetScannerId.java       |    5 +-
 .../client/TestAsyncAccessControlAdminApi.java     |   74 ++
 .../hbase/client/TestAsyncRegionAdminApi.java      |   21 +
 .../hbase/client/TestAsyncRegionAdminApi2.java     |   58 +-
 .../hbase/client/TestAsyncTableAdminApi.java       |    6 +-
 .../hbase/client/TestAsyncTableAdminApi2.java      |   10 -
 .../hbase/client/TestAsyncTableAdminApi3.java      |   18 +
 .../hadoop/hbase/client/TestClientTimeouts.java    |    2 +-
 .../hbase/client/TestConnectionImplementation.java |   16 +-
 .../hadoop/hbase/client/TestFromClientSide.java    |   16 +-
 .../hadoop/hbase/client/TestFromClientSide3.java   |   30 +-
 .../hbase/client/TestHBaseAdminNoCluster.java      |   20 +-
 .../hadoop/hbase/client/TestLeaseRenewal.java      |    9 +-
 .../hadoop/hbase/client/TestMetaWithReplicas.java  |    2 +-
 ...oneSnapshotFromClientCloneLinksAfterDelete.java |    5 +-
 .../hbase/client/TestMultiRespectsLimits.java      |    2 +-
 .../hbase/client/TestMultipleTimestamps.java       |    1 -
 .../hbase/client/TestReplicaWithCluster.java       |    8 +-
 ...estoreSnapshotFromClientWithRegionReplicas.java |   62 --
 .../hbase/client/TestScanWithoutFetchingData.java  |    5 +-
 .../client/TestSnapshotCloneIndependence.java      |   20 +-
 .../hbase/client/TestSnapshotFromClient.java       |   45 +-
 .../hadoop/hbase/client/TestSnapshotMetadata.java  |   21 +-
 .../client/TestSnapshotTemporaryDirectory.java     |   57 +-
 .../hbase/client/TestSplitOrMergeStatus.java       |   64 +-
 .../hadoop/hbase/client/TestTableFavoredNodes.java |   28 +-
 .../hbase/client/TestTableSnapshotScanner.java     |    3 +-
 .../TestReplicationAdminWithClusters.java          |   26 +-
 .../hbase/coprocessor/TestCoprocessorMetrics.java  |    2 +-
 .../hbase/coprocessor/TestMasterObserver.java      |   10 +-
 .../hbase/coprocessor/TestMetaTableMetrics.java    |   99 ++
 .../io/encoding/TestLoadAndSwitchEncodeOnDisk.java |    4 +-
 .../master/TestCatalogJanitorInMemoryStates.java   |   21 +-
 .../hbase/master/TestMasterBalanceThrottling.java  |    4 +-
 .../TestMasterOperationsForRegionReplicas.java     |    6 +-
 .../hbase/master/TestMasterStatusServlet.java      |   10 +-
 .../assignment/TestAssignmentManagerBase.java      |    7 +
 .../master/assignment/TestAssignmentOnRSCrash.java |    2 +-
 .../assignment/TestCloseRegionWhileRSCrash.java    |   14 +-
 .../assignment/TestOpenRegionProcedureHang.java    |  209 ++++
 .../master/assignment/TestRogueRSAssignment.java   |   16 +-
 .../balancer/TestFavoredNodeTableImport.java       |    9 +-
 .../TestFavoredStochasticBalancerPickers.java      |    2 +-
 .../TestFavoredStochasticLoadBalancer.java         |   50 +-
 .../master/cleaner/TestSnapshotFromMaster.java     |    2 +-
 .../hbase/master/locking/TestLockManager.java      |    8 +-
 .../TestSimpleRegionNormalizerOnCluster.java       |    8 +-
 .../procedure/TestCloneSnapshotProcedure.java      |    2 +-
 .../TestDeleteColumnFamilyProcedureFromClient.java |    6 +-
 .../TestMasterFailoverWithProcedures.java          |    2 +-
 .../master/procedure/TestModifyTableProcedure.java |   34 +-
 .../hbase/master/procedure/TestProcedureAdmin.java |    4 +-
 .../procedure/TestRestoreSnapshotProcedure.java    |    9 +-
 .../procedure/TestServerRemoteProcedure.java       |   16 +-
 .../procedure/TestTableDDLProcedureBase.java       |    8 +-
 .../TestTableDescriptorModificationFromClient.java |   10 +-
 .../hbase/mob/compactions/TestMobCompactor.java    |   11 +-
 .../hbase/namespace/TestNamespaceAuditor.java      |   10 +-
 .../hbase/procedure/TestProcedureManager.java      |    2 +-
 .../hadoop/hbase/quotas/TestQuotaThrottle.java     |  151 +++
 .../hadoop/hbase/quotas/TestSpaceQuotas.java       |  115 ++-
 .../hadoop/hbase/quotas/TestTablesWithQuotas.java  |    2 +-
 .../regionserver/TestCompactionFileNotFound.java   |    9 +-
 .../regionserver/TestEndToEndSplitTransaction.java |    3 +-
 .../TestRegionMergeTransactionOnCluster.java       |    6 +-
 .../hadoop/hbase/regionserver/TestRegionOpen.java  |    4 +-
 .../regionserver/TestRegionServerMetrics.java      |   14 -
 .../TestRegionReplicaReplicationEndpoint.java      |    6 +-
 .../hbase/security/access/SecureTestUtil.java      |   51 +-
 .../security/access/TestAccessController.java      |  183 ++--
 .../TestCoprocessorWhitelistMasterObserver.java    |   11 +-
 .../security/access/TestNamespaceCommands.java     |   33 +-
 .../security/access/TestPermissionBuilder.java     |   11 +-
 .../hbase/security/access/TestRpcAccessChecks.java |    2 +-
 .../security/access/TestTablePermissions.java      |  134 ++-
 .../access/TestWithDisabledAuthorization.java      |   24 +-
 .../security/access/TestZKPermissionWatcher.java   |    7 +-
 .../security/visibility/TestVisibilityLabels.java  |    2 +-
 .../hbase/snapshot/SnapshotTestingUtils.java       |    2 +-
 .../snapshot/TestFlushSnapshotFromClient.java      |   44 +-
 .../TestRestoreFlushSnapshotFromClient.java        |   20 +-
 .../hbase/snapshot/TestRestoreSnapshotHelper.java  |   42 +
 .../snapshot/TestSnapshotDescriptionUtils.java     |   50 +-
 .../apache/hadoop/hbase/util/TestHBaseFsckMOB.java |    2 +-
 .../hadoop/hbase/util/TestLossyCounting.java       |   10 +-
 .../hbase/util/TestMiniClusterLoadSequential.java  |    5 +-
 .../util/hbck/OfflineMetaRebuildTestCore.java      |   12 +-
 hbase-shell/src/main/ruby/hbase/admin.rb           |   91 +-
 hbase-shell/src/main/ruby/hbase/quotas.rb          |  179 +++-
 hbase-shell/src/main/ruby/hbase/security.rb        |    4 +-
 hbase-shell/src/main/ruby/hbase/taskmonitor.rb     |    2 +-
 hbase-shell/src/main/ruby/shell/commands/split.rb  |   11 +-
 .../hbase/thrift2/ThriftHBaseServiceHandler.java   |   10 +-
 .../hadoop/hbase/thrift2/client/ThriftAdmin.java   |  269 +----
 .../main/resources/hbase-webapps/thrift/thrift.jsp |    1 +
 hbase-zookeeper/pom.xml                            |    3 -
 .../hbase/zookeeper/RecoverableZooKeeper.java      |   23 +-
 .../org/apache/hadoop/hbase/zookeeper/ZKUtil.java  |   69 +-
 .../apache/hadoop/hbase/zookeeper/TestZKMulti.java |   95 +-
 pom.xml                                            |   95 +-
 src/main/asciidoc/_chapters/architecture.adoc      |   12 +-
 src/main/asciidoc/_chapters/datamodel.adoc         |    2 +-
 src/main/asciidoc/_chapters/hbase-default.adoc     |    6 +-
 src/main/asciidoc/_chapters/mapreduce.adoc         |    4 +-
 src/main/asciidoc/_chapters/ops_mgt.adoc           |   10 +-
 src/main/asciidoc/_chapters/profiler.adoc          |   98 ++
 src/main/asciidoc/_chapters/zookeeper.adoc         |    2 +-
 src/main/asciidoc/book.adoc                        |    1 +
 src/site/xdoc/downloads.xml                        |   37 +-
 261 files changed, 5116 insertions(+), 4481 deletions(-)
 create mode 100755 dev-support/hbase-vote.sh
 delete mode 100644 hbase-build-support/hbase-error-prone/pom.xml
 delete mode 100644 hbase-build-support/hbase-error-prone/src/main/java/org/apache/hadoop/hbase/errorprone/AlwaysPasses.java
 delete mode 100644 hbase-build-support/pom.xml
 create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/GetUserPermissionsRequest.java
 copy hbase-http/src/main/java/org/apache/hadoop/hbase/http/{AdminAuthorizedServlet.java => ProfileOutputServlet.java} (51%)
 create mode 100644 hbase-http/src/main/java/org/apache/hadoop/hbase/http/ProfileServlet.java
 create mode 100644 hbase-http/src/main/java/org/apache/hadoop/hbase/util/ProcessUtils.java
 create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAccessControlAdminApi.java
 delete mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClientWithRegionReplicas.java
 create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureHang.java
 create mode 100644 src/main/asciidoc/_chapters/profiler.adoc


[hbase] 40/49: HBASE-22102 Remove AsyncAdmin.isTableAvailable(TableName, byte[][])

Posted by op...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

openinx pushed a commit to branch HBASE-21879
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit 39f97946c5bce02c2b76ea80697009052a835e3b
Author: pingsutw <pi...@gmail.com>
AuthorDate: Tue Mar 26 21:26:23 2019 +0800

    HBASE-22102 Remove AsyncAdmin.isTableAvailable(TableName, byte[][])
    
    Signed-off-by: zhangduo <zh...@apache.org>
---
 .../main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java | 12 ------------
 .../java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java |  5 -----
 .../org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java   |  7 -------
 .../apache/hadoop/hbase/client/TestAsyncTableAdminApi.java   |  6 +++---
 .../apache/hadoop/hbase/client/TestAsyncTableAdminApi2.java  | 10 ----------
 5 files changed, 3 insertions(+), 37 deletions(-)

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 3227f22..ea75f60 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
@@ -222,18 +222,6 @@ public interface AsyncAdmin {
   CompletableFuture<Boolean> isTableAvailable(TableName tableName);
 
   /**
-   * Use this api to check if the table has been created with the specified number of splitkeys
-   * which was used while creating the given table. Note : If this api is used after a table's
-   * region gets splitted, the api may return false. The return value will be wrapped by a
-   * {@link CompletableFuture}.
-   * @param tableName name of table to check
-   * @param splitKeys keys to check if the table has been created with all split keys
-   * @deprecated Since 2.2.0. Will be removed in 3.0.0. Use {@link #isTableAvailable(TableName)}
-   */
-  @Deprecated
-  CompletableFuture<Boolean> isTableAvailable(TableName tableName, byte[][] splitKeys);
-
-  /**
    * Add a column family to an existing table.
    * @param tableName name of the table to add column family to
    * @param columnFamily column family descriptor of column family to be added
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 07b4311..668e729 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
@@ -177,11 +177,6 @@ class AsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<Boolean> isTableAvailable(TableName tableName, byte[][] splitKeys) {
-    return wrap(rawAdmin.isTableAvailable(tableName, splitKeys));
-  }
-
-  @Override
   public CompletableFuture<Void> addColumnFamily(TableName tableName,
       ColumnFamilyDescriptor columnFamily) {
     return wrap(rawAdmin.addColumnFamily(tableName, columnFamily));
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 085fcf7..248e732 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
@@ -687,13 +687,6 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
     return isTableAvailable(tableName, Optional.empty());
   }
 
-  @Override
-  public CompletableFuture<Boolean> isTableAvailable(TableName tableName, byte[][] splitKeys) {
-    Preconditions.checkNotNull(splitKeys, "splitKeys is null. If you don't specify splitKeys,"
-        + " use isTableAvailable(TableName) instead");
-    return isTableAvailable(tableName, Optional.of(splitKeys));
-  }
-
   private CompletableFuture<Boolean> isTableAvailable(TableName tableName,
       Optional<byte[][]> splitKeys) {
     if (TableName.isMetaTableName(tableName)) {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java
index 990eed4..e1eaca7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java
@@ -133,7 +133,7 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
     boolean tablesOnMaster = LoadBalancer.isTablesOnMaster(TEST_UTIL.getConfiguration());
     createTableWithDefaultConf(tableName, splitKeys);
 
-    boolean tableAvailable = admin.isTableAvailable(tableName, splitKeys).get();
+    boolean tableAvailable = admin.isTableAvailable(tableName).get();
     assertTrue("Table should be created with splitKyes + 1 rows in META", tableAvailable);
 
     AsyncTable<AdvancedScanResultConsumer> metaTable = ASYNC_CONN.getTable(META_TABLE_NAME);
@@ -406,7 +406,7 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
 
     assertEquals(NUM_REGIONS, TEST_UTIL.getHBaseCluster().getRegions(tableName).size());
     assertTrue("Table should be created with splitKyes + 1 rows in META",
-        admin.isTableAvailable(tableName, splitKeys).get());
+        admin.isTableAvailable(tableName).get());
 
     // Clone & Verify
     admin.cloneTableSchema(tableName, newTableName, preserveSplits).join();
@@ -421,7 +421,7 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
     if (preserveSplits) {
       assertEquals(NUM_REGIONS, TEST_UTIL.getHBaseCluster().getRegions(newTableName).size());
       assertTrue("New table should be created with splitKyes + 1 rows in META",
-          admin.isTableAvailable(newTableName, splitKeys).get());
+          admin.isTableAvailable(newTableName).get());
     } else {
       assertEquals(1, TEST_UTIL.getHBaseCluster().getRegions(newTableName).size());
     }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi2.java
index 63ff3e4..4f5cbf2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi2.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi2.java
@@ -209,16 +209,6 @@ public class TestAsyncTableAdminApi2 extends TestAsyncAdminBase {
     }
   }
 
-
-  @Test
-  public void testTableAvailableWithRandomSplitKeys() throws Exception {
-    createTableWithDefaultConf(tableName);
-    byte[][] splitKeys = new byte[1][];
-    splitKeys = new byte[][] { new byte[] { 1, 1, 1 }, new byte[] { 2, 2, 2 } };
-    boolean tableAvailable = admin.isTableAvailable(tableName, splitKeys).get();
-    assertFalse("Table should be created with 1 row in META", tableAvailable);
-  }
-
   @Test
   public void testCompactionTimestamps() throws Exception {
     createTableWithDefaultConf(tableName);


[hbase] 02/49: HBASE-21977 Skip replay WAL and update seqid when open regions restored from snapshot

Posted by op...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

openinx pushed a commit to branch HBASE-21879
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit eb44482ee0bd5b8a81531774c0b390cce2e74755
Author: meiyi <my...@gamil.com>
AuthorDate: Thu Mar 7 09:57:37 2019 +0800

    HBASE-21977 Skip replay WAL and update seqid when open regions restored from snapshot
    
    Signed-off-by: zhangduo <zh...@apache.org>
---
 .../hbase/client/ClientSideRegionScanner.java      |  1 +
 .../apache/hadoop/hbase/regionserver/HRegion.java  | 30 ++++++++++------
 .../hbase/snapshot/TestRestoreSnapshotHelper.java  | 42 ++++++++++++++++++++++
 3 files changed, 63 insertions(+), 10 deletions(-)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClientSideRegionScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClientSideRegionScanner.java
index 23a2399..a23cede 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClientSideRegionScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClientSideRegionScanner.java
@@ -60,6 +60,7 @@ public class ClientSideRegionScanner extends AbstractClientScanner {
     // open region from the snapshot directory
     region = HRegion.newHRegion(FSUtils.getTableDir(rootDir, htd.getTableName()), null, fs, conf,
       hri, htd, null);
+    region.setRestoredRegion(true);
     // we won't initialize the MobFileCache when not running in RS process. so provided an
     // initialized cache. Consider the case: an CF was set from an mob to non-mob. if we only
     // initialize cache for MOB region, NPE from HMobStore will still happen. So Initialize the
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index a553750..26ff24e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -340,6 +340,13 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   private Path regionDir;
   private FileSystem walFS;
 
+  // set to true if the region is restored from snapshot
+  private boolean isRestoredRegion = false;
+
+  public void setRestoredRegion(boolean restoredRegion) {
+    isRestoredRegion = restoredRegion;
+  }
+
   // The internal wait duration to acquire a lock before read/update
   // from the region. It is not per row. The purpose of this wait time
   // is to avoid waiting a long time while the region is busy, so that
@@ -953,7 +960,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     status.setStatus("Initializing all the Stores");
     long maxSeqId = initializeStores(reporter, status);
     this.mvcc.advanceTo(maxSeqId);
-    if (ServerRegionReplicaUtil.shouldReplayRecoveredEdits(this)) {
+    if (!isRestoredRegion && ServerRegionReplicaUtil.shouldReplayRecoveredEdits(this)) {
       Collection<HStore> stores = this.stores.values();
       try {
         // update the stores that we are replaying
@@ -1006,15 +1013,18 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
 
     // Use maximum of log sequenceid or that which was found in stores
     // (particularly if no recovered edits, seqid will be -1).
-    long maxSeqIdFromFile =
-        WALSplitter.getMaxRegionSequenceId(getWalFileSystem(), getWALRegionDirOfDefaultReplica());
-    long nextSeqId = Math.max(maxSeqId, maxSeqIdFromFile) + 1;
-    // The openSeqNum will always be increase even for read only region, as we rely on it to
-    // determine whether a region has been successfully reopend, so here we always need to update
-    // the max sequence id file.
-    if (RegionReplicaUtil.isDefaultReplica(getRegionInfo())) {
-      LOG.debug("writing seq id for {}", this.getRegionInfo().getEncodedName());
-      WALSplitter.writeRegionSequenceIdFile(getWalFileSystem(), getWALRegionDir(), nextSeqId - 1);
+    long nextSeqId = maxSeqId + 1;
+    if (!isRestoredRegion) {
+      long maxSeqIdFromFile =
+          WALSplitter.getMaxRegionSequenceId(getWalFileSystem(), getWALRegionDirOfDefaultReplica());
+      nextSeqId = Math.max(maxSeqId, maxSeqIdFromFile) + 1;
+      // The openSeqNum will always be increase even for read only region, as we rely on it to
+      // determine whether a region has been successfully reopend, so here we always need to update
+      // the max sequence id file.
+      if (RegionReplicaUtil.isDefaultReplica(getRegionInfo())) {
+        LOG.debug("writing seq id for {}", this.getRegionInfo().getEncodedName());
+        WALSplitter.writeRegionSequenceIdFile(getWalFileSystem(), getWALRegionDir(), nextSeqId - 1);
+      }
     }
 
     LOG.info("Opened {}; next sequenceid={}", this.getRegionInfo().getShortNameToLog(), nextSeqId);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreSnapshotHelper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreSnapshotHelper.java
index c1ce040..acc1f55 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreSnapshotHelper.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreSnapshotHelper.java
@@ -31,12 +31,14 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
 import org.apache.hadoop.hbase.io.HFileLink;
 import org.apache.hadoop.hbase.mob.MobUtils;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
+import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
 import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils.SnapshotMock;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -45,6 +47,7 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.FSTableDescriptors;
 import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.wal.WALSplitter;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Assert;
@@ -136,6 +139,45 @@ public class TestRestoreSnapshotHelper {
     checkNoHFileLinkInTableDir(tableName);
   }
 
+  @Test
+  public void testSkipReplayAndUpdateSeqId() throws Exception {
+    rootDir = TEST_UTIL.getDefaultRootDirPath();
+    FSUtils.setRootDir(conf, rootDir);
+    TableName tableName = TableName.valueOf("testSkipReplayAndUpdateSeqId");
+    String snapshotName = "testSkipReplayAndUpdateSeqId";
+    createTableAndSnapshot(tableName, snapshotName);
+    // put some data in the table
+    Table table = TEST_UTIL.getConnection().getTable(tableName);
+    TEST_UTIL.loadTable(table, Bytes.toBytes("A"));
+
+    Configuration conf = TEST_UTIL.getConfiguration();
+    Path rootDir = FSUtils.getRootDir(conf);
+    Path restoreDir = new Path("/hbase/.tmp-restore/testScannerWithRestoreScanner2");
+    // restore snapshot.
+    final RestoreSnapshotHelper.RestoreMetaChanges meta =
+        RestoreSnapshotHelper.copySnapshotForScanner(conf, fs, rootDir, restoreDir, snapshotName);
+    TableDescriptor htd = meta.getTableDescriptor();
+    final List<RegionInfo> restoredRegions = meta.getRegionsToAdd();
+    for (RegionInfo restoredRegion : restoredRegions) {
+      // open restored region
+      HRegion region = HRegion.newHRegion(FSUtils.getTableDir(restoreDir, tableName), null, fs,
+        conf, restoredRegion, htd, null);
+      // set restore flag
+      region.setRestoredRegion(true);
+      region.initialize();
+      Path recoveredEdit =
+          FSUtils.getWALRegionDir(conf, tableName, region.getRegionInfo().getEncodedName());
+      long maxSeqId = WALSplitter.getMaxRegionSequenceId(fs, recoveredEdit);
+
+      // open restored region without set restored flag
+      HRegion region2 = HRegion.newHRegion(FSUtils.getTableDir(restoreDir, tableName), null, fs,
+        conf, restoredRegion, htd, null);
+      region2.initialize();
+      long maxSeqId2 = WALSplitter.getMaxRegionSequenceId(fs, recoveredEdit);
+      Assert.assertTrue(maxSeqId2 > maxSeqId);
+    }
+  }
+
   protected void createTableAndSnapshot(TableName tableName, String snapshotName)
       throws IOException {
     byte[] column = Bytes.toBytes("A");


[hbase] 45/49: HBASE-22092 Typo in block cache monitoring documentation

Posted by op...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

openinx pushed a commit to branch HBASE-21879
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit a5caeb519d3be06abc3eff07b18853e2ae584a37
Author: William Shen <wi...@users.noreply.github.com>
AuthorDate: Fri Mar 22 15:11:13 2019 -0700

    HBASE-22092 Typo in block cache monitoring documentation
---
 src/main/asciidoc/_chapters/ops_mgt.adoc | 10 +++++-----
 1 file changed, 5 insertions(+), 5 deletions(-)

diff --git a/src/main/asciidoc/_chapters/ops_mgt.adoc b/src/main/asciidoc/_chapters/ops_mgt.adoc
index ee7bd97..b4bdb21 100644
--- a/src/main/asciidoc/_chapters/ops_mgt.adoc
+++ b/src/main/asciidoc/_chapters/ops_mgt.adoc
@@ -1794,19 +1794,19 @@ This particular example, for example, would indicate that the likely cause of sl
 === Block Cache Monitoring
 
 Starting with HBase 0.98, the HBase Web UI includes the ability to monitor and report on the performance of the block cache.
-To view the block cache reports, click .
+To view the block cache reports, see the Block Cache section of the region server UI.
 Following are a few examples of the reporting capabilities.
 
-.Basic Info
+.Basic Info shows the cache implementation.
 image::bc_basic.png[]
 
-.Config
+.Config shows all cache configuration options.
 image::bc_config.png[]
 
-.Stats
+.Stats shows statistics about the performance of the cache.
 image::bc_stats.png[]
 
-.L1 and L2
+.L1 and L2 shows information about the L1 and L2 caches.
 image::bc_l1.png[]
 
 This is not an exhaustive list of all the screens and reports available.


[hbase] 15/49: HBASE-22015 UserPermission should be annotated as InterfaceAudience.Public

Posted by op...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

openinx pushed a commit to branch HBASE-21879
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit c43b7c8616758378af2a905990e6183ee5ff774b
Author: meiyi <my...@gamil.com>
AuthorDate: Mon Mar 18 11:43:07 2019 +0800

    HBASE-22015 UserPermission should be annotated as InterfaceAudience.Public
    
    Signed-off-by: Guanghao Zhang <zg...@apache.org>
---
 .../java/org/apache/hadoop/hbase/client/Admin.java |  13 +-
 .../org/apache/hadoop/hbase/client/AsyncAdmin.java |  13 +-
 .../hadoop/hbase/client/AsyncHBaseAdmin.java       |  10 +-
 .../org/apache/hadoop/hbase/client/HBaseAdmin.java |  12 +-
 .../hadoop/hbase/client/RawAsyncHBaseAdmin.java    |  20 ++-
 .../hbase/security/access/AccessControlClient.java |  20 +--
 .../hbase/security/access/AccessControlUtil.java   |  47 ++++----
 .../hbase/security/access/GlobalPermission.java    |  17 +--
 .../hbase/security/access/NamespacePermission.java |  21 +---
 .../hadoop/hbase/security/access/Permission.java   |  32 ++++-
 .../security/access/ShadedAccessControlUtil.java   |  31 +++--
 .../hbase/security/access/TablePermission.java     |  67 +----------
 .../hbase/security/access/UserPermission.java      |  85 +------------
 .../hbase/security/access/AccessControlLists.java  |   9 +-
 .../hbase/security/access/AccessController.java    |  17 +--
 .../hbase/snapshot/RestoreSnapshotHelper.java      |   6 +-
 .../hbase/security/access/SecureTestUtil.java      |  45 ++++---
 .../security/access/TestAccessController.java      |  49 ++++----
 .../security/access/TestNamespaceCommands.java     |  18 +--
 .../security/access/TestPermissionBuilder.java     |  11 +-
 .../security/access/TestTablePermissions.java      | 134 +++++++++++++--------
 .../access/TestWithDisabledAuthorization.java      |  24 ++--
 .../security/access/TestZKPermissionWatcher.java   |   7 +-
 .../hadoop/hbase/thrift2/client/ThriftAdmin.java   |   6 +-
 24 files changed, 302 insertions(+), 412 deletions(-)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
index e595e51..4da5f6f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
@@ -53,7 +53,7 @@ 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.replication.SyncReplicationState;
-import org.apache.hadoop.hbase.security.access.Permission;
+import org.apache.hadoop.hbase.security.access.UserPermission;
 import org.apache.hadoop.hbase.snapshot.HBaseSnapshotException;
 import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
 import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
@@ -2038,21 +2038,18 @@ public interface Admin extends Abortable, Closeable {
 
   /**
    * Grants user specific permissions
-   * @param userName user name
-   * @param permission the specific permission
+   * @param userPermission user name and the specific permission
    * @param mergeExistingPermissions If set to false, later granted permissions will override
    *          previous granted permissions. otherwise, it'll merge with previous granted
    *          permissions.
    * @throws IOException if a remote or network exception occurs
    */
-  void grant(String userName, Permission permission, boolean mergeExistingPermissions)
-      throws IOException;
+  void grant(UserPermission userPermission, boolean mergeExistingPermissions) throws IOException;
 
   /**
    * Revokes user specific permissions
-   * @param userName user name
-   * @param permission the specific permission
+   * @param userPermission user name and the specific permission
    * @throws IOException if a remote or network exception occurs
    */
-  void revoke(String userName, Permission permission) throws IOException;
+  void revoke(UserPermission userPermission) throws IOException;
 }
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 a4994e6..07807ed 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
@@ -46,7 +46,7 @@ import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshotView;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.replication.SyncReplicationState;
-import org.apache.hadoop.hbase.security.access.Permission;
+import org.apache.hadoop.hbase.security.access.UserPermission;
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
@@ -1428,19 +1428,16 @@ public interface AsyncAdmin {
 
   /**
    * Grants user specific permissions
-   * @param userName user name
-   * @param permission the specific permission
+   * @param userPermission user name and the specific permission
    * @param mergeExistingPermissions If set to false, later granted permissions will override
    *          previous granted permissions. otherwise, it'll merge with previous granted
    *          permissions.
    */
-  CompletableFuture<Void> grant(String userName, Permission permission,
-      boolean mergeExistingPermissions);
+  CompletableFuture<Void> grant(UserPermission userPermission, boolean mergeExistingPermissions);
 
   /**
    * Revokes user specific permissions
-   * @param userName user name
-   * @param permission the specific permission
+   * @param userPermission user name and the specific permission
    */
-  CompletableFuture<Void> revoke(String userName, Permission permission);
+  CompletableFuture<Void> revoke(UserPermission userPermission);
 }
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 41095d0..038f94e 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
@@ -42,7 +42,7 @@ import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.replication.SyncReplicationState;
-import org.apache.hadoop.hbase.security.access.Permission;
+import org.apache.hadoop.hbase.security.access.UserPermission;
 import org.apache.hadoop.hbase.util.FutureUtils;
 import org.apache.yetus.audience.InterfaceAudience;
 
@@ -798,13 +798,13 @@ class AsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<Void> grant(String userName, Permission permission,
+  public CompletableFuture<Void> grant(UserPermission userPermission,
       boolean mergeExistingPermissions) {
-    return wrap(rawAdmin.grant(userName, permission, mergeExistingPermissions));
+    return wrap(rawAdmin.grant(userPermission, mergeExistingPermissions));
   }
 
   @Override
-  public CompletableFuture<Void> revoke(String userName, Permission permission) {
-    return wrap(rawAdmin.revoke(userName, permission));
+  public CompletableFuture<Void> revoke(UserPermission userPermission) {
+    return wrap(rawAdmin.revoke(userPermission));
   }
 }
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 02ba1d0..51e126c 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
@@ -85,7 +85,6 @@ import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.replication.SyncReplicationState;
-import org.apache.hadoop.hbase.security.access.Permission;
 import org.apache.hadoop.hbase.security.access.ShadedAccessControlUtil;
 import org.apache.hadoop.hbase.security.access.UserPermission;
 import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
@@ -3819,13 +3818,13 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public void grant(String userName, Permission permission, boolean mergeExistingPermissions)
+  public void grant(UserPermission userPermission, boolean mergeExistingPermissions)
       throws IOException {
     executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
       @Override
       protected Void rpcCall() throws Exception {
-        GrantRequest req = ShadedAccessControlUtil
-            .buildGrantRequest(new UserPermission(userName, permission), mergeExistingPermissions);
+        GrantRequest req =
+            ShadedAccessControlUtil.buildGrantRequest(userPermission, mergeExistingPermissions);
         this.master.grant(getRpcController(), req);
         return null;
       }
@@ -3833,12 +3832,11 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public void revoke(String userName, Permission permission) throws IOException {
+  public void revoke(UserPermission userPermission) throws IOException {
     executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
       @Override
       protected Void rpcCall() throws Exception {
-        RevokeRequest req =
-            ShadedAccessControlUtil.buildRevokeRequest(new UserPermission(userName, permission));
+        RevokeRequest req = ShadedAccessControlUtil.buildRevokeRequest(userPermission);
         this.master.revoke(getRpcController(), req);
         return null;
       }
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 7a97ce8..498ef49 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
@@ -85,7 +85,6 @@ 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.replication.SyncReplicationState;
-import org.apache.hadoop.hbase.security.access.Permission;
 import org.apache.hadoop.hbase.security.access.ShadedAccessControlUtil;
 import org.apache.hadoop.hbase.security.access.UserPermission;
 import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
@@ -3756,24 +3755,21 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<Void> grant(String userName, Permission permission,
+  public CompletableFuture<Void> grant(UserPermission userPermission,
       boolean mergeExistingPermissions) {
     return this.<Void> newMasterCaller()
-        .action(
-          (controller, stub) -> this.<GrantRequest, GrantResponse, Void> call(controller, stub,
-            ShadedAccessControlUtil.buildGrantRequest(new UserPermission(userName, permission),
-              mergeExistingPermissions),
-            (s, c, req, done) -> s.grant(c, req, done), resp -> null))
+        .action((controller, stub) -> this.<GrantRequest, GrantResponse, Void> call(controller,
+          stub, ShadedAccessControlUtil.buildGrantRequest(userPermission, mergeExistingPermissions),
+          (s, c, req, done) -> s.grant(c, req, done), resp -> null))
         .call();
   }
 
   @Override
-  public CompletableFuture<Void> revoke(String userName, Permission permission) {
+  public CompletableFuture<Void> revoke(UserPermission userPermission) {
     return this.<Void> newMasterCaller()
-        .action(
-          (controller, stub) -> this.<RevokeRequest, RevokeResponse, Void> call(controller, stub,
-            ShadedAccessControlUtil.buildRevokeRequest(new UserPermission(userName, permission)),
-            (s, c, req, done) -> s.revoke(c, req, done), resp -> null))
+        .action((controller, stub) -> this.<RevokeRequest, RevokeResponse, Void> call(controller,
+          stub, ShadedAccessControlUtil.buildRevokeRequest(userPermission),
+          (s, c, req, done) -> s.revoke(c, req, done), resp -> null))
         .call();
   }
 }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlClient.java
index 1b1fe43..c467dfb 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlClient.java
@@ -92,7 +92,8 @@ public class AccessControlClient {
   private static void grant(Connection connection, final TableName tableName,
       final String userName, final byte[] family, final byte[] qual, boolean mergeExistingPermissions,
       final Permission.Action... actions) throws Throwable {
-    connection.getAdmin().grant(userName, new TablePermission(tableName, family, qual, actions),
+    connection.getAdmin().grant(new UserPermission(userName, Permission.newBuilder(tableName)
+        .withFamily(family).withQualifier(qual).withActions(actions).build()),
       mergeExistingPermissions);
   }
 
@@ -125,7 +126,8 @@ public class AccessControlClient {
    */
   private static void grant(Connection connection, final String namespace, final String userName,
       boolean mergeExistingPermissions, final Permission.Action... actions) throws Throwable {
-    connection.getAdmin().grant(userName, new NamespacePermission(namespace, actions),
+    connection.getAdmin().grant(
+      new UserPermission(userName, Permission.newBuilder(namespace).withActions(actions).build()),
       mergeExistingPermissions);
   }
 
@@ -156,7 +158,9 @@ public class AccessControlClient {
    */
   private static void grant(Connection connection, final String userName,
       boolean mergeExistingPermissions, final Permission.Action... actions) throws Throwable {
-    connection.getAdmin().grant(userName, new GlobalPermission(actions), mergeExistingPermissions);
+    connection.getAdmin().grant(
+      new UserPermission(userName, Permission.newBuilder().withActions(actions).build()),
+      mergeExistingPermissions);
   }
 
   /**
@@ -193,8 +197,8 @@ public class AccessControlClient {
   public static void revoke(Connection connection, final TableName tableName,
       final String username, final byte[] family, final byte[] qualifier,
       final Permission.Action... actions) throws Throwable {
-    connection.getAdmin().revoke(username,
-      new TablePermission(tableName, family, qualifier, actions));
+    connection.getAdmin().revoke(new UserPermission(username, Permission.newBuilder(tableName)
+        .withFamily(family).withQualifier(qualifier).withActions(actions).build()));
   }
 
   /**
@@ -207,7 +211,8 @@ public class AccessControlClient {
    */
   public static void revoke(Connection connection, final String namespace,
       final String userName, final Permission.Action... actions) throws Throwable {
-    connection.getAdmin().revoke(userName, new NamespacePermission(namespace, actions));
+    connection.getAdmin().revoke(
+      new UserPermission(userName, Permission.newBuilder(namespace).withActions(actions).build()));
   }
 
   /**
@@ -216,7 +221,8 @@ public class AccessControlClient {
    */
   public static void revoke(Connection connection, final String userName,
       final Permission.Action... actions) throws Throwable {
-    connection.getAdmin().revoke(userName, new GlobalPermission(actions));
+    connection.getAdmin()
+        .revoke(new UserPermission(userName, Permission.newBuilder().withActions(actions).build()));
   }
 
   /**
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlUtil.java
index 216b332..05f173e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlUtil.java
@@ -247,7 +247,7 @@ public class AccessControlUtil {
    * @return the converted TablePermission
    */
   public static TablePermission toTablePermission(AccessControlProtos.TablePermission proto) {
-    List<Permission.Action> actions = toPermissionActions(proto.getActionList());
+    Permission.Action[] actions = toPermissionActions(proto.getActionList());
     TableName table = null;
     byte[] qualifier = null;
     byte[] family = null;
@@ -261,8 +261,7 @@ public class AccessControlUtil {
     if (proto.hasQualifier()) {
       qualifier = proto.getQualifier().toByteArray();
     }
-    return new TablePermission(table, family, qualifier,
-        actions.toArray(new Permission.Action[actions.size()]));
+    return new TablePermission(table, family, qualifier, actions);
   }
 
   /**
@@ -273,21 +272,21 @@ public class AccessControlUtil {
   public static Permission toPermission(AccessControlProtos.Permission proto) {
     if (proto.getType() == AccessControlProtos.Permission.Type.Global) {
       AccessControlProtos.GlobalPermission perm = proto.getGlobalPermission();
-      List<Permission.Action> actions = toPermissionActions(perm.getActionList());
-      return new GlobalPermission(actions.toArray(new Permission.Action[actions.size()]));
+      Permission.Action[] actions = toPermissionActions(perm.getActionList());
+      return Permission.newBuilder().withActions(actions).build();
     }
     if (proto.getType() == AccessControlProtos.Permission.Type.Namespace) {
       AccessControlProtos.NamespacePermission perm = proto.getNamespacePermission();
-      List<Permission.Action> actions = toPermissionActions(perm.getActionList());
+      Permission.Action[] actions = toPermissionActions(perm.getActionList());
       if (!proto.hasNamespacePermission()) {
         throw new IllegalStateException("Namespace must not be empty in NamespacePermission");
       }
-      return new NamespacePermission(perm.getNamespaceName().toStringUtf8(),
-        actions.toArray(new Permission.Action[actions.size()]));
+      return Permission.newBuilder(perm.getNamespaceName().toStringUtf8()).withActions(actions)
+          .build();
     }
     if (proto.getType() == AccessControlProtos.Permission.Type.Table) {
       AccessControlProtos.TablePermission perm = proto.getTablePermission();
-      List<Permission.Action> actions = toPermissionActions(perm.getActionList());
+      Permission.Action[] actions = toPermissionActions(perm.getActionList());
       byte[] qualifier = null;
       byte[] family = null;
       TableName table = null;
@@ -301,8 +300,8 @@ public class AccessControlUtil {
       if (perm.hasQualifier()) {
         qualifier = perm.getQualifier().toByteArray();
       }
-      return new TablePermission(table, family, qualifier,
-        actions.toArray(new Permission.Action[actions.size()]));
+      return Permission.newBuilder(table).withFamily(family).withQualifier(qualifier)
+          .withActions(actions).build();
     }
     throw new IllegalStateException("Unrecognize Perm Type: " + proto.getType());
   }
@@ -364,16 +363,16 @@ public class AccessControlUtil {
   }
 
   /**
-   * Converts a list of Permission.Action proto to a list of client Permission.Action objects.
+   * Converts a list of Permission.Action proto to an array of client Permission.Action objects.
    *
    * @param protoActions the list of protobuf Actions
-   * @return the converted list of Actions
+   * @return the converted array of Actions
    */
-  public static List<Permission.Action> toPermissionActions(
-      List<AccessControlProtos.Permission.Action> protoActions) {
-    List<Permission.Action> actions = new ArrayList<>(protoActions.size());
-    for (AccessControlProtos.Permission.Action a : protoActions) {
-      actions.add(toPermissionAction(a));
+  public static Permission.Action[]
+      toPermissionActions(List<AccessControlProtos.Permission.Action> protoActions) {
+    Permission.Action[] actions = new Permission.Action[protoActions.size()];
+    for (int i = 0; i < protoActions.size(); i++) {
+      actions[i] = toPermissionAction(protoActions.get(i));
     }
     return actions;
   }
@@ -491,7 +490,7 @@ public class AccessControlUtil {
    * @param userShortName the short name of the user to grant permissions
    * @param actions the permissions to be granted
    * @throws ServiceException
-   * @deprecated Use {@link Admin#grant(String, Permission, boolean)} instead.
+   * @deprecated Use {@link Admin#grant(UserPermission, boolean)} instead.
    */
   @Deprecated
   public static void grant(RpcController controller,
@@ -520,7 +519,7 @@ public class AccessControlUtil {
    * @param q optional qualifier
    * @param actions the permissions to be granted
    * @throws ServiceException
-   * @deprecated Use {@link Admin#grant(String, Permission, boolean)} instead.
+   * @deprecated Use {@link Admin#grant(UserPermission, boolean)} instead.
    */
   @Deprecated
   public static void grant(RpcController controller,
@@ -548,7 +547,7 @@ public class AccessControlUtil {
    * @param namespace the short name of the user to grant permissions
    * @param actions the permissions to be granted
    * @throws ServiceException
-   * @deprecated Use {@link Admin#grant(String, Permission, boolean)} instead.
+   * @deprecated Use {@link Admin#grant(UserPermission, boolean)} instead.
    */
   @Deprecated
   public static void grant(RpcController controller,
@@ -574,7 +573,7 @@ public class AccessControlUtil {
    * @param userShortName the short name of the user to revoke permissions
    * @param actions the permissions to be revoked
    * @throws ServiceException on failure
-   * @deprecated Use {@link Admin#revoke(String, Permission)} instead.
+   * @deprecated Use {@link Admin#revoke(UserPermission)} instead.
    */
   @Deprecated
   public static void revoke(RpcController controller,
@@ -604,7 +603,7 @@ public class AccessControlUtil {
    * @param q optional qualifier
    * @param actions the permissions to be revoked
    * @throws ServiceException on failure
-   * @deprecated Use {@link Admin#revoke(String, Permission)} instead.
+   * @deprecated Use {@link Admin#revoke(UserPermission)} instead.
    */
   @Deprecated
   public static void revoke(RpcController controller,
@@ -631,7 +630,7 @@ public class AccessControlUtil {
    * @param namespace optional table name
    * @param actions the permissions to be revoked
    * @throws ServiceException on failure
-   * @deprecated Use {@link Admin#revoke(String, Permission)} instead.
+   * @deprecated Use {@link Admin#revoke(UserPermission)} instead.
    */
   @Deprecated
   public static void revoke(RpcController controller,
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/GlobalPermission.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/GlobalPermission.java
index b29317a..01d53eb 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/GlobalPermission.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/GlobalPermission.java
@@ -23,15 +23,9 @@ import org.apache.yetus.audience.InterfaceAudience;
 /**
  * Represents an authorization for access whole cluster.
  */
-@InterfaceAudience.Private
+@InterfaceAudience.Public
 public class GlobalPermission extends Permission {
 
-  /** Default constructor for Writable, do not use */
-  public GlobalPermission() {
-    super();
-    this.scope = Scope.EMPTY;
-  }
-
   /**
    * Construct a global permission.
    * @param assigned assigned actions
@@ -41,15 +35,6 @@ public class GlobalPermission extends Permission {
     this.scope = Scope.GLOBAL;
   }
 
-  /**
-   * Construct a global permission.
-   * @param actionCode assigned actions
-   */
-  GlobalPermission(byte[] actionCode) {
-    super(actionCode);
-    this.scope = Scope.GLOBAL;
-  }
-
   @Override
   public int hashCode() {
     return super.hashCode();
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/NamespacePermission.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/NamespacePermission.java
index c7ede96..7781d22 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/NamespacePermission.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/NamespacePermission.java
@@ -30,39 +30,22 @@ import org.apache.yetus.audience.InterfaceAudience;
 /**
  * Represents an authorization for access for the given namespace.
  */
-@InterfaceAudience.Private
+@InterfaceAudience.Public
 public class NamespacePermission extends Permission {
 
   private String namespace = NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR;
 
-  /** Default constructor for Writable, do not use */
-  public NamespacePermission() {
-    super();
-    this.scope = Scope.EMPTY;
-  }
-
   /**
    * Construct a namespace permission.
    * @param namespace namespace's name
    * @param assigned assigned actions
    */
-  public NamespacePermission(String namespace, Action... assigned) {
+  NamespacePermission(String namespace, Action... assigned) {
     super(assigned);
     this.namespace = namespace;
     this.scope = Scope.NAMESPACE;
   }
 
-  /**
-   * Construct a namespace permission.
-   * @param namespace namespace's name
-   * @param actionCode assigned actions
-   */
-  public NamespacePermission(String namespace, byte[] actionCode) {
-    super(actionCode);
-    this.namespace = namespace;
-    this.scope = Scope.NAMESPACE;
-  }
-
   public String getNamespace() {
     return namespace;
   }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/Permission.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/Permission.java
index e80cd33..49f2432 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/Permission.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/Permission.java
@@ -21,8 +21,10 @@ package org.apache.hadoop.hbase.security.access;
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.EnumSet;
+import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 
@@ -275,7 +277,7 @@ public class Permission extends VersionedWritable {
     private TableName tableName;
     private byte[] family;
     private byte[] qualifier;
-    private Action[] actions;
+    private List<Action> actions = new ArrayList<>();
 
     private Builder() {
     }
@@ -301,17 +303,37 @@ public class Permission extends VersionedWritable {
     }
 
     public Builder withActions(Action... actions) {
-      this.actions = actions;
+      for (Action action : actions) {
+        if (action != null) {
+          this.actions.add(action);
+        }
+      }
+      return this;
+    }
+
+    public Builder withActionCodes(byte[] actionCodes) {
+      if (actionCodes != null) {
+        for (byte code : actionCodes) {
+          Action action = ACTION_BY_CODE.get(code);
+          if (action == null) {
+            LOG.error("Ignoring unknown action code '{}'",
+              Bytes.toStringBinary(new byte[] { code }));
+            continue;
+          }
+          this.actions.add(action);
+        }
+      }
       return this;
     }
 
     public Permission build() {
+      Action[] actionArray = actions.toArray(new Action[actions.size()]);
       if (namespace != null) {
-        return new NamespacePermission(namespace, actions);
+        return new NamespacePermission(namespace, actionArray);
       } else if (tableName != null) {
-        return new TablePermission(tableName, family, qualifier, actions);
+        return new TablePermission(tableName, family, qualifier, actionArray);
       } else {
-        return new GlobalPermission(actions);
+        return new GlobalPermission(actionArray);
       }
     }
   }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/ShadedAccessControlUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/ShadedAccessControlUtil.java
index 67fdba3..b354d87 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/ShadedAccessControlUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/ShadedAccessControlUtil.java
@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.hbase.security.access;
 
-import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 import java.util.Map;
@@ -86,16 +85,16 @@ public class ShadedAccessControlUtil {
   }
 
   /**
-   * Converts a list of Permission.Action shaded proto to a list of client Permission.Action
+   * Converts a list of Permission.Action shaded proto to an array of client Permission.Action
    * objects.
    * @param protoActions the list of shaded protobuf Actions
-   * @return the converted list of Actions
+   * @return the converted array of Actions
    */
-  public static List<Permission.Action> toPermissionActions(
-      List<AccessControlProtos.Permission.Action> protoActions) {
-    List<Permission.Action> actions = new ArrayList<>(protoActions.size());
-    for (AccessControlProtos.Permission.Action a : protoActions) {
-      actions.add(toPermissionAction(a));
+  public static Permission.Action[]
+      toPermissionActions(List<AccessControlProtos.Permission.Action> protoActions) {
+    Permission.Action[] actions = new Permission.Action[protoActions.size()];
+    for (int i = 0; i < protoActions.size(); i++) {
+      actions[i] = toPermissionAction(protoActions.get(i));
     }
     return actions;
   }
@@ -121,23 +120,22 @@ public class ShadedAccessControlUtil {
 
     if (proto.getType() == AccessControlProtos.Permission.Type.Global) {
       AccessControlProtos.GlobalPermission perm = proto.getGlobalPermission();
-      List<Action> actions = toPermissionActions(perm.getActionList());
-
-      return new GlobalPermission(actions.toArray(new Permission.Action[actions.size()]));
+      Action[] actions = toPermissionActions(perm.getActionList());
+      return Permission.newBuilder().withActions(actions).build();
     }
     if (proto.getType() == AccessControlProtos.Permission.Type.Namespace) {
       AccessControlProtos.NamespacePermission perm = proto.getNamespacePermission();
-      List<Permission.Action> actions = toPermissionActions(perm.getActionList());
+      Action[] actions = toPermissionActions(perm.getActionList());
 
       if (!proto.hasNamespacePermission()) {
         throw new IllegalStateException("Namespace must not be empty in NamespacePermission");
       }
       String ns = perm.getNamespaceName().toStringUtf8();
-      return new NamespacePermission(ns, actions.toArray(new Permission.Action[actions.size()]));
+      return Permission.newBuilder(ns).withActions(actions).build();
     }
     if (proto.getType() == AccessControlProtos.Permission.Type.Table) {
       AccessControlProtos.TablePermission perm = proto.getTablePermission();
-      List<Permission.Action> actions = toPermissionActions(perm.getActionList());
+      Action[] actions = toPermissionActions(perm.getActionList());
 
       byte[] qualifier = null;
       byte[] family = null;
@@ -149,9 +147,8 @@ public class ShadedAccessControlUtil {
 
       if (perm.hasFamily()) family = perm.getFamily().toByteArray();
       if (perm.hasQualifier()) qualifier = perm.getQualifier().toByteArray();
-
-      return new TablePermission(table, family, qualifier,
-          actions.toArray(new Permission.Action[actions.size()]));
+      return Permission.newBuilder(table).withFamily(family).withQualifier(qualifier)
+          .withActions(actions).build();
     }
     throw new IllegalStateException("Unrecognize Perm Type: " + proto.getType());
   }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/TablePermission.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/TablePermission.java
index 36ed8e4..f17919f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/TablePermission.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/TablePermission.java
@@ -34,38 +34,13 @@ import org.apache.hadoop.hbase.util.Bytes;
  * given table. If the family property is <code>null</code>, it implies
  * full table access.
  */
-@InterfaceAudience.Private
+@InterfaceAudience.Public
 public class TablePermission extends Permission {
 
   private TableName table;
   private byte[] family;
   private byte[] qualifier;
 
-  /** Nullary constructor for Writable, do not use */
-  public TablePermission() {
-    super();
-    this.scope = Scope.EMPTY;
-  }
-
-  /**
-   * Construct a table permission.
-   * @param table table name
-   * @param assigned assigned actions
-   */
-  public TablePermission(TableName table, Action... assigned) {
-    this(table, null, null, assigned);
-  }
-
-  /**
-   * Construct a table:family permission.
-   * @param table table name
-   * @param family family name
-   * @param assigned assigned actions
-   */
-  public TablePermission(TableName table, byte[] family, Action... assigned) {
-    this(table, family, null, assigned);
-  }
-
   /**
    * Construct a table:family:qualifier permission.
    * @param table table name
@@ -73,7 +48,7 @@ public class TablePermission extends Permission {
    * @param qualifier qualifier name
    * @param assigned assigned actions
    */
-  public TablePermission(TableName table, byte[] family, byte[] qualifier, Action... assigned) {
+  TablePermission(TableName table, byte[] family, byte[] qualifier, Action... assigned) {
     super(assigned);
     this.table = table;
     this.family = family;
@@ -81,48 +56,10 @@ public class TablePermission extends Permission {
     this.scope = Scope.TABLE;
   }
 
-  /**
-   * Construct a table permission.
-   * @param table table name
-   * @param actionCodes assigned actions
-   */
-  public TablePermission(TableName table, byte[] actionCodes) {
-    this(table, null, null, actionCodes);
-  }
-
-  /**
-   * Construct a table:family permission.
-   * @param table table name
-   * @param family family name
-   * @param actionCodes assigned actions
-   */
-  public TablePermission(TableName table, byte[] family, byte[] actionCodes) {
-    this(table, family, null, actionCodes);
-  }
-
-  /**
-   * Construct a table:family:qualifier permission.
-   * @param table table name
-   * @param family family name
-   * @param qualifier qualifier name
-   * @param actionCodes assigned actions
-   */
-  public TablePermission(TableName table, byte[] family, byte[] qualifier, byte[] actionCodes) {
-    super(actionCodes);
-    this.table = table;
-    this.family = family;
-    this.qualifier = qualifier;
-    this.scope = Scope.TABLE;
-  }
-
   public TableName getTableName() {
     return table;
   }
 
-  public void setTableName(TableName table) {
-    this.table = table;
-  }
-
   public boolean hasFamily() {
     return family != null;
   }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/UserPermission.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/UserPermission.java
index 2a9a109..896ba52 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/UserPermission.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/UserPermission.java
@@ -20,102 +20,19 @@ package org.apache.hadoop.hbase.security.access;
 
 import java.util.Objects;
 
-import org.apache.hadoop.hbase.TableName;
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * UserPermission consists of a user name and a permission.
  * Permission can be one of [Global, Namespace, Table] permission.
  */
-@InterfaceAudience.Private
+@InterfaceAudience.Public
 public class UserPermission {
 
   private String user;
   private Permission permission;
 
   /**
-   * Construct a global user permission.
-   * @param user user name
-   * @param assigned assigned actions
-   */
-  public UserPermission(String user, Permission.Action... assigned) {
-    this.user = user;
-    this.permission = new GlobalPermission(assigned);
-  }
-
-  /**
-   * Construct a global user permission.
-   * @param user user name
-   * @param actionCode action codes
-   */
-  public UserPermission(String user, byte[] actionCode) {
-    this.user = user;
-    this.permission = new GlobalPermission(actionCode);
-  }
-
-  /**
-   * Construct a namespace user permission.
-   * @param user user name
-   * @param namespace namespace
-   * @param assigned assigned actions
-   */
-  public UserPermission(String user, String namespace, Permission.Action... assigned) {
-    this.user = user;
-    this.permission = new NamespacePermission(namespace, assigned);
-  }
-
-  /**
-   * Construct a table user permission.
-   * @param user user name
-   * @param tableName table name
-   * @param assigned assigned actions
-   */
-  public UserPermission(String user, TableName tableName, Permission.Action... assigned) {
-    this.user = user;
-    this.permission = new TablePermission(tableName, assigned);
-  }
-
-  /**
-   * Construct a table:family user permission.
-   * @param user user name
-   * @param tableName table name
-   * @param family family name of table
-   * @param assigned assigned actions
-   */
-  public UserPermission(String user, TableName tableName, byte[] family,
-    Permission.Action... assigned) {
-    this(user, tableName, family, null, assigned);
-  }
-
-  /**
-   * Construct a table:family:qualifier user permission.
-   * @param user user name
-   * @param tableName table name
-   * @param family family name of table
-   * @param qualifier qualifier name of table
-   * @param assigned assigned actions
-   */
-  public UserPermission(String user, TableName tableName, byte[] family, byte[] qualifier,
-      Permission.Action... assigned) {
-    this.user = user;
-    this.permission = new TablePermission(tableName, family, qualifier, assigned);
-  }
-
-  /**
-   * Construct a table:family:qualifier user permission.
-   * @param user user name
-   * @param tableName table name
-   * @param family family name of table
-   * @param qualifier qualifier name of table
-   * @param actionCodes assigned actions
-   */
-  public UserPermission(String user, TableName tableName, byte[] family, byte[] qualifier,
-      byte[] actionCodes) {
-    this.user = user;
-    this.permission = new TablePermission(tableName, family, qualifier, actionCodes);
-  }
-
-  /**
    * Construct a user permission given permission.
    * @param user user name
    * @param permission one of [Global, Namespace, Table] permission
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java
index b5bf3a4..74c848d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java
@@ -638,7 +638,8 @@ public class AccessControlLists {
       }
 
       return new Pair<>(username,
-        new NamespacePermission(Bytes.toString(fromNamespaceEntry(entryName)), value));
+          Permission.newBuilder(Bytes.toString(fromNamespaceEntry(entryName)))
+              .withActionCodes(value).build());
     }
 
     // Handle global entry
@@ -648,7 +649,7 @@ public class AccessControlLists {
         return null;
       }
 
-      return new Pair<>(username, new GlobalPermission(value));
+      return new Pair<>(username, Permission.newBuilder().withActionCodes(value).build());
     }
 
     // Handle table entry
@@ -681,8 +682,8 @@ public class AccessControlLists {
       }
     }
 
-    return new Pair<>(username,
-        new TablePermission(TableName.valueOf(entryName), permFamily, permQualifier, value));
+    return new Pair<>(username, Permission.newBuilder(TableName.valueOf(entryName))
+        .withFamily(permFamily).withQualifier(permQualifier).withActionCodes(value).build());
   }
 
   /*
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
index 2898a71..a4a96c0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
@@ -887,7 +887,7 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor,
         if (owner == null)
           owner = getActiveUser(c).getShortName();
         final UserPermission userPermission = new UserPermission(owner,
-          desc.getTableName(), Action.values());
+            Permission.newBuilder(desc.getTableName()).withActions(Action.values()).build());
         // switch to the real hbase master user for doing the RPC on the ACL table
         User.runAsLoginUser(new PrivilegedExceptionAction<Void>() {
           @Override
@@ -990,7 +990,7 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor,
       @Override
       public Void run() throws Exception {
         UserPermission userperm = new UserPermission(owner,
-          currentDesc.getTableName(), Action.values());
+            Permission.newBuilder(currentDesc.getTableName()).withActions(Action.values()).build());
         try (Table table = c.getEnvironment().getConnection().
             getTable(AccessControlLists.ACL_TABLE_NAME)) {
           AccessControlLists.addUserPermission(conf, userperm, table);
@@ -2053,7 +2053,7 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor,
   /* ---- Protobuf AccessControlService implementation ---- */
 
   /**
-   * @deprecated Use {@link Admin#grant(String, Permission, boolean)} instead.
+   * @deprecated Use {@link Admin#grant(UserPermission, boolean)} instead.
    */
   @Deprecated
   @Override
@@ -2076,7 +2076,8 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor,
         preGrantOrRevoke(caller, "grant", perm);
 
         // regionEnv is set at #start. Hopefully not null at this point.
-        regionEnv.getConnection().getAdmin().grant(perm.getUser(), perm.getPermission(),
+        regionEnv.getConnection().getAdmin().grant(
+          new UserPermission(perm.getUser(), perm.getPermission()),
           request.getMergeExistingPermissions());
         if (AUDITLOG.isTraceEnabled()) {
           // audit log should store permission changes in addition to auth results
@@ -2095,7 +2096,7 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor,
   }
 
   /**
-   * @deprecated Use {@link Admin#revoke(String, Permission)} instead.
+   * @deprecated Use {@link Admin#revoke(UserPermission)} instead.
    */
   @Deprecated
   @Override
@@ -2116,7 +2117,8 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor,
         }
         preGrantOrRevoke(caller, "revoke", perm);
         // regionEnv is set at #start. Hopefully not null here.
-        regionEnv.getConnection().getAdmin().revoke(perm.getUser(), perm.getPermission());
+        regionEnv.getConnection().getAdmin()
+            .revoke(new UserPermission(perm.getUser(), perm.getPermission()));
         if (AUDITLOG.isTraceEnabled()) {
           // audit log should record all permission changes
           AUDITLOG.trace("Revoked permission " + perm.toString());
@@ -2211,7 +2213,8 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor,
             // them. Also using acl as table name to be inline with the results of global admin and
             // will help in avoiding any leakage of information about being superusers.
             for (String user : Superusers.getSuperUsers()) {
-              perms.add(new UserPermission(user, Action.values()));
+              perms.add(new UserPermission(user,
+                  Permission.newBuilder().withActions(Action.values()).build()));
             }
           }
         }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java
index 0acfb1a..ec251b8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java
@@ -865,10 +865,8 @@ public class RestoreSnapshotHelper {
         for (Entry<String, Permission> e : perms.entries()) {
           String user = e.getKey();
           TablePermission tablePerm = (TablePermission) e.getValue();
-          TablePermission newPerm = new TablePermission(newTableName,
-            tablePerm.getFamily(), tablePerm.getQualifier(), tablePerm.getActions());
-          AccessControlClient.grant(conn, newPerm.getTableName(), user, newPerm.getFamily(),
-            newPerm.getQualifier(), newPerm.getActions());
+          AccessControlClient.grant(conn, newTableName, user, tablePerm.getFamily(),
+            tablePerm.getQualifier(), tablePerm.getActions());
         }
       } catch (Throwable e) {
         throw new IOException("Grant acl into newly creatd table failed. snapshot: " + snapshot
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java
index ef448ce..129f2b6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java
@@ -378,7 +378,8 @@ public class SecureTestUtil {
       @Override
       public Void call() throws Exception {
         try (Connection connection = ConnectionFactory.createConnection(util.getConfiguration())) {
-          connection.getAdmin().grant(user, new GlobalPermission(actions), false);
+          connection.getAdmin().grant(
+            new UserPermission(user, Permission.newBuilder().withActions(actions).build()), false);
         }
         return null;
       }
@@ -397,8 +398,8 @@ public class SecureTestUtil {
       public Void call() throws Exception {
         Configuration conf = util.getConfiguration();
         try (Connection connection = ConnectionFactory.createConnection(conf, caller)) {
-          connection.getAdmin().grant(user, Permission.newBuilder().withActions(actions).build(),
-            false);
+          connection.getAdmin().grant(
+            new UserPermission(user, Permission.newBuilder().withActions(actions).build()), false);
         }
         return null;
       }
@@ -416,7 +417,8 @@ public class SecureTestUtil {
       @Override
       public Void call() throws Exception {
         try (Connection connection = ConnectionFactory.createConnection(util.getConfiguration())) {
-          connection.getAdmin().revoke(user, new GlobalPermission(actions));
+          connection.getAdmin().revoke(
+            new UserPermission(user, Permission.newBuilder().withActions(actions).build()));
         }
         return null;
       }
@@ -435,7 +437,8 @@ public class SecureTestUtil {
       public Void call() throws Exception {
         Configuration conf = util.getConfiguration();
         try (Connection connection = ConnectionFactory.createConnection(conf, caller)) {
-          connection.getAdmin().revoke(user, Permission.newBuilder().withActions(actions).build());
+          connection.getAdmin().revoke(
+            new UserPermission(user, Permission.newBuilder().withActions(actions).build()));
         }
         return null;
       }
@@ -453,8 +456,9 @@ public class SecureTestUtil {
       @Override
       public Void call() throws Exception {
         try (Connection connection = ConnectionFactory.createConnection(util.getConfiguration())) {
-          connection.getAdmin()
-              .grant(user, new NamespacePermission(namespace, actions), false);
+          connection.getAdmin().grant(
+            new UserPermission(user, Permission.newBuilder(namespace).withActions(actions).build()),
+            false);
         }
         return null;
       }
@@ -474,8 +478,9 @@ public class SecureTestUtil {
       public Void call() throws Exception {
         Configuration conf = util.getConfiguration();
         try (Connection connection = ConnectionFactory.createConnection(conf, caller)) {
-          connection.getAdmin().grant(user,
-            Permission.newBuilder(namespace).withActions(actions).build(), false);
+          connection.getAdmin().grant(
+            new UserPermission(user, Permission.newBuilder(namespace).withActions(actions).build()),
+            false);
         }
         return null;
       }
@@ -535,7 +540,8 @@ public class SecureTestUtil {
       @Override
       public Void call() throws Exception {
         try (Connection connection = ConnectionFactory.createConnection(util.getConfiguration())) {
-          connection.getAdmin().revoke(user, new NamespacePermission(namespace, actions));
+          connection.getAdmin().revoke(new UserPermission(user,
+              Permission.newBuilder(namespace).withActions(actions).build()));
         }
         return null;
       }
@@ -555,8 +561,8 @@ public class SecureTestUtil {
       public Void call() throws Exception {
         Configuration conf = util.getConfiguration();
         try (Connection connection = ConnectionFactory.createConnection(conf, caller)) {
-          connection.getAdmin().revoke(user,
-            Permission.newBuilder(namespace).withActions(actions).build());
+          connection.getAdmin().revoke(new UserPermission(user,
+              Permission.newBuilder(namespace).withActions(actions).build()));
         }
         return null;
       }
@@ -575,7 +581,8 @@ public class SecureTestUtil {
       @Override
       public Void call() throws Exception {
         try (Connection connection = ConnectionFactory.createConnection(util.getConfiguration())) {
-          connection.getAdmin().grant(user, new TablePermission(table, family, qualifier, actions),
+          connection.getAdmin().grant(new UserPermission(user, Permission.newBuilder(table)
+              .withFamily(family).withQualifier(qualifier).withActions(actions).build()),
             false);
         }
         return null;
@@ -596,8 +603,8 @@ public class SecureTestUtil {
       public Void call() throws Exception {
         Configuration conf = util.getConfiguration();
         try (Connection connection = ConnectionFactory.createConnection(conf, caller)) {
-          connection.getAdmin().grant(user, Permission.newBuilder(table).withFamily(family)
-              .withQualifier(qualifier).withActions(actions).build(),
+          connection.getAdmin().grant(new UserPermission(user, Permission.newBuilder(table)
+              .withFamily(family).withQualifier(qualifier).withActions(actions).build()),
             false);
         }
         return null;
@@ -659,8 +666,8 @@ public class SecureTestUtil {
       @Override
       public Void call() throws Exception {
         try (Connection connection = ConnectionFactory.createConnection(util.getConfiguration())) {
-          connection.getAdmin().revoke(user,
-            new TablePermission(table, family, qualifier, actions));
+          connection.getAdmin().revoke(new UserPermission(user, Permission.newBuilder(table)
+              .withFamily(family).withQualifier(qualifier).withActions(actions).build()));
         }
         return null;
       }
@@ -680,8 +687,8 @@ public class SecureTestUtil {
       public Void call() throws Exception {
         Configuration conf = util.getConfiguration();
         try (Connection connection = ConnectionFactory.createConnection(conf, caller)) {
-          connection.getAdmin().revoke(user, Permission.newBuilder(table).withFamily(family)
-              .withQualifier(qualifier).withActions(actions).build());
+          connection.getAdmin().revoke(new UserPermission(user, Permission.newBuilder(table)
+              .withFamily(family).withQualifier(qualifier).withActions(actions).build()));
         }
         return null;
       }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
index ad97790..0d7a0e0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
@@ -1171,8 +1171,9 @@ public class TestAccessController extends SecureTestUtil {
       @Override
       public Object run() throws Exception {
         try (Connection conn = ConnectionFactory.createConnection(conf)) {
-          conn.getAdmin().grant(USER_RO.getShortName(),
-            new TablePermission(TEST_TABLE, TEST_FAMILY, Action.READ), false);
+          conn.getAdmin().grant(new UserPermission(USER_RO.getShortName(), Permission
+              .newBuilder(TEST_TABLE).withFamily(TEST_FAMILY).withActions(Action.READ).build()),
+            false);
         }
         return null;
       }
@@ -1182,8 +1183,8 @@ public class TestAccessController extends SecureTestUtil {
       @Override
       public Object run() throws Exception {
         try (Connection conn = ConnectionFactory.createConnection(conf)) {
-          conn.getAdmin().revoke(USER_RO.getShortName(), Permission.newBuilder(TEST_TABLE)
-              .withFamily(TEST_FAMILY).withActions(Action.READ).build());
+          conn.getAdmin().revoke(new UserPermission(USER_RO.getShortName(), Permission
+              .newBuilder(TEST_TABLE).withFamily(TEST_FAMILY).withActions(Action.READ).build()));
         }
         return null;
       }
@@ -1221,8 +1222,8 @@ public class TestAccessController extends SecureTestUtil {
       @Override
       public Object run() throws Exception {
         ACCESS_CONTROLLER.preGrant(ObserverContextImpl.createAndPrepare(CP_ENV),
-          new UserPermission(USER_RO.getShortName(),
-              new TablePermission(TEST_TABLE, TEST_FAMILY, Action.READ)),
+          new UserPermission(USER_RO.getShortName(), Permission.newBuilder(TEST_TABLE)
+              .withFamily(TEST_FAMILY).withActions(Action.READ).build()),
           false);
         return null;
       }
@@ -1232,8 +1233,8 @@ public class TestAccessController extends SecureTestUtil {
       @Override
       public Object run() throws Exception {
         ACCESS_CONTROLLER.preRevoke(ObserverContextImpl.createAndPrepare(CP_ENV),
-          new UserPermission(USER_RO.getShortName(),
-              new TablePermission(TEST_TABLE, TEST_FAMILY, Action.READ)));
+          new UserPermission(USER_RO.getShortName(), Permission.newBuilder(TEST_TABLE)
+              .withFamily(TEST_FAMILY).withActions(Action.READ).build()));
         return null;
       }
     };
@@ -1692,8 +1693,8 @@ public class TestAccessController extends SecureTestUtil {
         acl.close();
       }
 
-      UserPermission ownerperm =
-          new UserPermission(USER_OWNER.getName(), tableName, Action.values());
+      UserPermission ownerperm = new UserPermission(USER_OWNER.getName(),
+          Permission.newBuilder(tableName).withActions(Action.values()).build());
       assertTrue("Owner should have all permissions on table",
         hasFoundUserPermission(ownerperm, perms));
 
@@ -1701,7 +1702,8 @@ public class TestAccessController extends SecureTestUtil {
       String userName = user.getShortName();
 
       UserPermission up =
-          new UserPermission(userName, tableName, family1, qualifier, Permission.Action.READ);
+          new UserPermission(userName, Permission.newBuilder(tableName).withFamily(family1)
+              .withQualifier(qualifier).withActions(Permission.Action.READ).build());
       assertFalse("User should not be granted permission: " + up.toString(),
         hasFoundUserPermission(up, perms));
 
@@ -1720,12 +1722,13 @@ public class TestAccessController extends SecureTestUtil {
       }
 
       UserPermission upToVerify =
-          new UserPermission(userName, tableName, family1, qualifier, Permission.Action.READ);
+          new UserPermission(userName, Permission.newBuilder(tableName).withFamily(family1)
+              .withQualifier(qualifier).withActions(Permission.Action.READ).build());
       assertTrue("User should be granted permission: " + upToVerify.toString(),
         hasFoundUserPermission(upToVerify, perms));
 
-      upToVerify =
-          new UserPermission(userName, tableName, family1, qualifier, Permission.Action.WRITE);
+      upToVerify = new UserPermission(userName, Permission.newBuilder(tableName).withFamily(family1)
+          .withQualifier(qualifier).withActions(Permission.Action.WRITE).build());
       assertFalse("User should not be granted permission: " + upToVerify.toString(),
         hasFoundUserPermission(upToVerify, perms));
 
@@ -1743,9 +1746,9 @@ public class TestAccessController extends SecureTestUtil {
         acl.close();
       }
 
-      upToVerify =
-          new UserPermission(userName, tableName, family1, qualifier, Permission.Action.WRITE,
-              Permission.Action.READ);
+      upToVerify = new UserPermission(userName,
+          Permission.newBuilder(tableName).withFamily(family1).withQualifier(qualifier)
+              .withActions(Permission.Action.WRITE, Permission.Action.READ).build());
       assertTrue("User should be granted permission: " + upToVerify.toString(),
         hasFoundUserPermission(upToVerify, perms));
 
@@ -1783,8 +1786,8 @@ public class TestAccessController extends SecureTestUtil {
         acl.close();
       }
 
-      UserPermission newOwnerperm =
-          new UserPermission(newOwner.getName(), tableName, Action.values());
+      UserPermission newOwnerperm = new UserPermission(newOwner.getName(),
+          Permission.newBuilder(tableName).withActions(Action.values()).build());
       assertTrue("New owner should have all permissions on table",
         hasFoundUserPermission(newOwnerperm, perms));
     } finally {
@@ -1808,10 +1811,12 @@ public class TestAccessController extends SecureTestUtil {
 
     Collection<String> superUsers = Superusers.getSuperUsers();
     List<UserPermission> adminPerms = new ArrayList<>(superUsers.size() + 1);
-    adminPerms.add(new UserPermission(USER_ADMIN.getShortName(), Bytes.toBytes("ACRW")));
-    for(String user: superUsers) {
+    adminPerms.add(new UserPermission(USER_ADMIN.getShortName(), Permission.newBuilder()
+        .withActions(Action.ADMIN, Action.CREATE, Action.READ, Action.WRITE).build()));
+    for (String user : superUsers) {
       // Global permission
-      adminPerms.add(new UserPermission(user, Action.values()));
+      adminPerms.add(
+        new UserPermission(user, Permission.newBuilder().withActions(Action.values()).build()));
     }
     assertTrue("Only super users, global users and user admin has permission on table hbase:acl " +
         "per setup", perms.size() == 5 + superUsers.size() &&
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java
index 04e8092..82d0f6e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java
@@ -363,8 +363,9 @@ public class TestNamespaceCommands extends SecureTestUtil {
       @Override
       public Object run() throws Exception {
         try (Connection connection = ConnectionFactory.createConnection(conf)) {
-          connection.getAdmin().grant(testUser,
-            new NamespacePermission(TEST_NAMESPACE, Action.WRITE), false);
+          connection.getAdmin().grant(new UserPermission(testUser,
+              Permission.newBuilder(TEST_NAMESPACE).withActions(Action.WRITE).build()),
+            false);
         }
         return null;
       }
@@ -373,8 +374,9 @@ public class TestNamespaceCommands extends SecureTestUtil {
       @Override
       public Object run() throws Exception {
         try (Connection conn = ConnectionFactory.createConnection(conf)) {
-          conn.getAdmin().grant(USER_GROUP_NS_ADMIN.getShortName(),
-            new NamespacePermission(TEST_NAMESPACE, Action.READ), false);
+          conn.getAdmin().grant(new UserPermission(USER_GROUP_NS_ADMIN.getShortName(),
+              Permission.newBuilder(TEST_NAMESPACE).withActions(Action.READ).build()),
+            false);
         }
         return null;
       }
@@ -384,8 +386,8 @@ public class TestNamespaceCommands extends SecureTestUtil {
       @Override
       public Object run() throws Exception {
         try (Connection connection = ConnectionFactory.createConnection(conf)) {
-          connection.getAdmin().revoke(testUser,
-            new NamespacePermission(TEST_NAMESPACE, Action.WRITE));
+          connection.getAdmin().revoke(new UserPermission(testUser,
+              Permission.newBuilder(TEST_NAMESPACE).withActions(Action.WRITE).build()));
         }
         return null;
       }
@@ -394,8 +396,8 @@ public class TestNamespaceCommands extends SecureTestUtil {
       @Override
       public Object run() throws Exception {
         try (Connection connection = ConnectionFactory.createConnection(conf)) {
-          connection.getAdmin().revoke(USER_GROUP_NS_ADMIN.getShortName(),
-            new NamespacePermission(TEST_NAMESPACE, Action.READ));
+          connection.getAdmin().revoke(new UserPermission(USER_GROUP_NS_ADMIN.getShortName(),
+              Permission.newBuilder(TEST_NAMESPACE).withActions(Action.READ).build()));
         }
         return null;
       }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestPermissionBuilder.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestPermissionBuilder.java
index 74a0c62..ff002b1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestPermissionBuilder.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestPermissionBuilder.java
@@ -46,7 +46,7 @@ public class TestPermissionBuilder {
     assertEquals(0, permission.getActions().length);
 
     // check global permission with ADMIN action
-    permission = Permission.newBuilder().withActions(Action.ADMIN).build();
+    permission = Permission.newBuilder().withActionCodes(Bytes.toBytes("A")).build();
     assertTrue(permission instanceof GlobalPermission);
     assertEquals(1, permission.getActions().length);
     assertTrue(permission.getActions()[0] == Action.ADMIN);
@@ -57,8 +57,15 @@ public class TestPermissionBuilder {
           .withActions(Action.CREATE, Action.READ).build();
       fail("Should throw NPE");
     } catch (NullPointerException e) {
-      // catch NPE because set family but table name is null
+      // catch NPE because set qualifier but table name is null
     }
+
+    permission = Permission.newBuilder().withActionCodes(Bytes.toBytes("ACP"))
+        .withActions(Action.READ, Action.ADMIN).build();
+    assertEquals(3, permission.getActions().length);
+    assertEquals(Action.READ, permission.getActions()[0]);
+    assertEquals(Action.CREATE, permission.getActions()[1]);
+    assertEquals(Action.ADMIN, permission.getActions()[2]);
   }
 
   @Test
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java
index 1c478b2..8363665 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java
@@ -143,14 +143,18 @@ public class TestTablePermissions {
     try (Connection connection = ConnectionFactory.createConnection(conf)) {
       // add some permissions
       addUserPermission(conf,
-        new UserPermission("george", TEST_TABLE, Permission.Action.READ, Permission.Action.WRITE),
+        new UserPermission("george",
+            Permission.newBuilder(TEST_TABLE)
+                .withActions(Permission.Action.READ, Permission.Action.WRITE).build()),
         connection.getTable(AccessControlLists.ACL_TABLE_NAME));
       addUserPermission(conf,
-        new UserPermission("hubert", TEST_TABLE, Permission.Action.READ),
+        new UserPermission("hubert",
+            Permission.newBuilder(TEST_TABLE).withActions(Permission.Action.READ).build()),
         connection.getTable(AccessControlLists.ACL_TABLE_NAME));
       addUserPermission(conf,
-        new UserPermission("humphrey", TEST_TABLE, TEST_FAMILY, TEST_QUALIFIER,
-          Permission.Action.READ),
+        new UserPermission("humphrey",
+            Permission.newBuilder(TEST_TABLE).withFamily(TEST_FAMILY).withQualifier(TEST_QUALIFIER)
+                .withActions(Permission.Action.READ).build()),
         connection.getTable(AccessControlLists.ACL_TABLE_NAME));
     }
     // retrieve the same
@@ -211,7 +215,8 @@ public class TestTablePermissions {
     try (Connection connection = ConnectionFactory.createConnection(conf);
         Table table = connection.getTable(AccessControlLists.ACL_TABLE_NAME)) {
       AccessControlLists.addUserPermission(conf,
-        new UserPermission("hubert", TEST_TABLE2, Permission.Action.READ, Permission.Action.WRITE),
+        new UserPermission("hubert", Permission.newBuilder(TEST_TABLE2)
+            .withActions(Permission.Action.READ, Permission.Action.WRITE).build()),
         table);
     }
     // check full load
@@ -246,17 +251,24 @@ public class TestTablePermissions {
     Configuration conf = UTIL.getConfiguration();
     try (Connection connection = ConnectionFactory.createConnection(conf)) {
       addUserPermission(conf,
-        new UserPermission("albert", TEST_TABLE, Permission.Action.READ),
-          connection.getTable(AccessControlLists.ACL_TABLE_NAME));
+        new UserPermission("albert",
+            Permission.newBuilder(TEST_TABLE).withActions(Permission.Action.READ).build()),
+        connection.getTable(AccessControlLists.ACL_TABLE_NAME));
       addUserPermission(conf,
-        new UserPermission("betty", TEST_TABLE, Permission.Action.READ, Permission.Action.WRITE),
-          connection.getTable(AccessControlLists.ACL_TABLE_NAME));
+        new UserPermission("betty",
+            Permission.newBuilder(TEST_TABLE)
+                .withActions(Permission.Action.READ, Permission.Action.WRITE).build()),
+        connection.getTable(AccessControlLists.ACL_TABLE_NAME));
       addUserPermission(conf,
-        new UserPermission("clark", TEST_TABLE, TEST_FAMILY, Permission.Action.READ),
-          connection.getTable(AccessControlLists.ACL_TABLE_NAME));
+        new UserPermission("clark",
+            Permission.newBuilder(TEST_TABLE).withFamily(TEST_FAMILY)
+                .withActions(Permission.Action.READ).build()),
+        connection.getTable(AccessControlLists.ACL_TABLE_NAME));
       addUserPermission(conf,
-        new UserPermission("dwight", TEST_TABLE, TEST_FAMILY, TEST_QUALIFIER,
-          Permission.Action.WRITE), connection.getTable(AccessControlLists.ACL_TABLE_NAME));
+        new UserPermission("dwight",
+            Permission.newBuilder(TEST_TABLE).withFamily(TEST_FAMILY).withQualifier(TEST_QUALIFIER)
+                .withActions(Permission.Action.WRITE).build()),
+        connection.getTable(AccessControlLists.ACL_TABLE_NAME));
     }
     // verify permissions survive changes in table metadata
     ListMultimap<String, UserPermission> preperms =
@@ -302,17 +314,16 @@ public class TestTablePermissions {
 
   private ListMultimap<String, UserPermission> createPermissions() {
     ListMultimap<String, UserPermission> permissions = ArrayListMultimap.create();
-    permissions.put("george",
-      new UserPermission("george", TEST_TABLE, Permission.Action.READ));
-    permissions.put("george",
-      new UserPermission("george", TEST_TABLE, TEST_FAMILY, Permission.Action.WRITE));
-    permissions.put("george",
-      new UserPermission("george", TEST_TABLE2, Permission.Action.READ));
-    permissions.put("hubert",
-      new UserPermission("hubert", TEST_TABLE2, Permission.Action.READ,
-        Permission.Action.WRITE));
-    permissions.put("bruce",
-      new UserPermission("bruce", TEST_NAMESPACE, Permission.Action.READ));
+    permissions.put("george", new UserPermission("george",
+        Permission.newBuilder(TEST_TABLE).withActions(Permission.Action.READ).build()));
+    permissions.put("george", new UserPermission("george", Permission.newBuilder(TEST_TABLE)
+        .withFamily(TEST_FAMILY).withActions(Permission.Action.WRITE).build()));
+    permissions.put("george", new UserPermission("george",
+        Permission.newBuilder(TEST_TABLE2).withActions(Permission.Action.READ).build()));
+    permissions.put("hubert", new UserPermission("hubert", Permission.newBuilder(TEST_TABLE2)
+        .withActions(Permission.Action.READ, Permission.Action.WRITE).build()));
+    permissions.put("bruce", new UserPermission("bruce",
+        Permission.newBuilder(TEST_NAMESPACE).withActions(Permission.Action.READ).build()));
     return permissions;
   }
 
@@ -334,50 +345,58 @@ public class TestTablePermissions {
 
   @Test
   public void testEquals() throws Exception {
-    Permission p1 = new TablePermission(TEST_TABLE, Permission.Action.READ);
-    Permission p2 = new TablePermission(TEST_TABLE, Permission.Action.READ);
+    Permission p1 = Permission.newBuilder(TEST_TABLE).withActions(Permission.Action.READ).build();
+    Permission p2 = Permission.newBuilder(TEST_TABLE).withActions(Permission.Action.READ).build();
     assertTrue(p1.equals(p2));
     assertTrue(p2.equals(p1));
 
-    p1 = new TablePermission(TEST_TABLE, TablePermission.Action.READ, TablePermission.Action.WRITE);
-    p2 = new TablePermission(TEST_TABLE, TablePermission.Action.WRITE, TablePermission.Action.READ);
+    p1 = Permission.newBuilder(TEST_TABLE)
+        .withActions(TablePermission.Action.READ, TablePermission.Action.WRITE).build();
+    p2 = Permission.newBuilder(TEST_TABLE)
+        .withActions(TablePermission.Action.WRITE, TablePermission.Action.READ).build();
     assertTrue(p1.equals(p2));
     assertTrue(p2.equals(p1));
 
-    p1 = new TablePermission(TEST_TABLE, TEST_FAMILY, TablePermission.Action.READ, TablePermission.Action.WRITE);
-    p2 = new TablePermission(TEST_TABLE, TEST_FAMILY, TablePermission.Action.WRITE, TablePermission.Action.READ);
+    p1 = Permission.newBuilder(TEST_TABLE).withFamily(TEST_FAMILY)
+        .withActions(TablePermission.Action.READ, TablePermission.Action.WRITE).build();
+    p2 = Permission.newBuilder(TEST_TABLE).withFamily(TEST_FAMILY)
+        .withActions(TablePermission.Action.WRITE, TablePermission.Action.READ).build();
     assertTrue(p1.equals(p2));
     assertTrue(p2.equals(p1));
 
-    p1 = new TablePermission(TEST_TABLE, TEST_FAMILY, TEST_QUALIFIER, TablePermission.Action.READ, TablePermission.Action.WRITE);
-    p2 = new TablePermission(TEST_TABLE, TEST_FAMILY, TEST_QUALIFIER, TablePermission.Action.WRITE, TablePermission.Action.READ);
+    p1 = Permission.newBuilder(TEST_TABLE).withFamily(TEST_FAMILY).withQualifier(TEST_QUALIFIER)
+        .withActions(TablePermission.Action.READ, TablePermission.Action.WRITE).build();
+    p2 = Permission.newBuilder(TEST_TABLE).withFamily(TEST_FAMILY).withQualifier(TEST_QUALIFIER)
+        .withActions(TablePermission.Action.WRITE, TablePermission.Action.READ).build();
     assertTrue(p1.equals(p2));
     assertTrue(p2.equals(p1));
 
-    p1 = new TablePermission(TEST_TABLE, TablePermission.Action.READ);
-    p2 = new TablePermission(TEST_TABLE, TEST_FAMILY, TablePermission.Action.READ);
+    p1 = Permission.newBuilder(TEST_TABLE).withActions(TablePermission.Action.READ).build();
+    p2 = Permission.newBuilder(TEST_TABLE).withFamily(TEST_FAMILY)
+        .withActions(TablePermission.Action.READ).build();
     assertFalse(p1.equals(p2));
     assertFalse(p2.equals(p1));
 
-    p1 = new TablePermission(TEST_TABLE, TablePermission.Action.READ);
-    p2 = new TablePermission(TEST_TABLE, TablePermission.Action.WRITE);
+    p1 = Permission.newBuilder(TEST_TABLE).withActions(TablePermission.Action.READ).build();
+    p2 = Permission.newBuilder(TEST_TABLE).withActions(TablePermission.Action.WRITE).build();
     assertFalse(p1.equals(p2));
     assertFalse(p2.equals(p1));
-    p2 = new TablePermission(TEST_TABLE, TablePermission.Action.READ, TablePermission.Action.WRITE);
+    p2 = Permission.newBuilder(TEST_TABLE)
+        .withActions(TablePermission.Action.READ, TablePermission.Action.WRITE).build();
     assertFalse(p1.equals(p2));
     assertFalse(p2.equals(p1));
 
-    p1 = new TablePermission(TEST_TABLE, TablePermission.Action.READ);
-    p2 = new TablePermission(TEST_TABLE2, TablePermission.Action.READ);
+    p1 = Permission.newBuilder(TEST_TABLE).withActions(TablePermission.Action.READ).build();
+    p2 = Permission.newBuilder(TEST_TABLE2).withActions(TablePermission.Action.READ).build();
     assertFalse(p1.equals(p2));
     assertFalse(p2.equals(p1));
 
-    p1 = new NamespacePermission(TEST_NAMESPACE, TablePermission.Action.READ);
-    p2 = new NamespacePermission(TEST_NAMESPACE, TablePermission.Action.READ);
+    p1 = Permission.newBuilder(TEST_NAMESPACE).withActions(TablePermission.Action.READ).build();
+    p2 = Permission.newBuilder(TEST_NAMESPACE).withActions(TablePermission.Action.READ).build();
     assertEquals(p1, p2);
 
-    p1 = new NamespacePermission(TEST_NAMESPACE, TablePermission.Action.READ);
-    p2 = new NamespacePermission(TEST_NAMESPACE2, TablePermission.Action.READ);
+    p1 = Permission.newBuilder(TEST_NAMESPACE).withActions(TablePermission.Action.READ).build();
+    p2 = Permission.newBuilder(TEST_NAMESPACE2).withActions(TablePermission.Action.READ).build();
     assertFalse(p1.equals(p2));
     assertFalse(p2.equals(p1));
   }
@@ -389,15 +408,20 @@ public class TestTablePermissions {
     // add some permissions
     try (Connection connection = ConnectionFactory.createConnection(conf)) {
       addUserPermission(conf,
-          new UserPermission("user1",
-              Permission.Action.READ, Permission.Action.WRITE), connection.getTable(AccessControlLists.ACL_TABLE_NAME));
+        new UserPermission("user1", Permission.newBuilder()
+            .withActions(Permission.Action.READ, Permission.Action.WRITE).build()),
+        connection.getTable(AccessControlLists.ACL_TABLE_NAME));
       addUserPermission(conf,
-          new UserPermission("user2",
-              Permission.Action.CREATE), connection.getTable(AccessControlLists.ACL_TABLE_NAME));
+        new UserPermission("user2",
+            Permission.newBuilder().withActions(Permission.Action.CREATE).build()),
+        connection.getTable(AccessControlLists.ACL_TABLE_NAME));
       addUserPermission(conf,
-          new UserPermission("user3",
-              Permission.Action.ADMIN, Permission.Action.READ, Permission.Action.CREATE),
-          connection.getTable(AccessControlLists.ACL_TABLE_NAME));
+        new UserPermission("user3",
+            Permission.newBuilder()
+                .withActions(Permission.Action.ADMIN, Permission.Action.READ,
+                  Permission.Action.CREATE)
+                .build()),
+        connection.getTable(AccessControlLists.ACL_TABLE_NAME));
     }
     ListMultimap<String, UserPermission> perms =
       AccessControlLists.getTablePermissions(conf, null);
@@ -434,9 +458,13 @@ public class TestTablePermissions {
     User currentUser = User.getCurrent();
     assertTrue(authManager.authorizeUserGlobal(currentUser, Permission.Action.ADMIN));
     try (Connection connection = ConnectionFactory.createConnection(conf)) {
-      for (int i=1; i<=50; i++) {
-        addUserPermission(conf, new UserPermission("testauth"+i,
-          Permission.Action.ADMIN, Permission.Action.READ, Permission.Action.WRITE),
+      for (int i = 1; i <= 50; i++) {
+        addUserPermission(conf,
+          new UserPermission("testauth" + i,
+              Permission.newBuilder()
+                  .withActions(Permission.Action.ADMIN, Permission.Action.READ,
+                    Permission.Action.WRITE)
+                  .build()),
           connection.getTable(AccessControlLists.ACL_TABLE_NAME));
         // make sure the system user still shows as authorized
         assertTrue("Failed current user auth check on iter "+i,
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestWithDisabledAuthorization.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestWithDisabledAuthorization.java
index 67c43ee..bc59f70 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestWithDisabledAuthorization.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestWithDisabledAuthorization.java
@@ -381,11 +381,12 @@ public class TestWithDisabledAuthorization extends SecureTestUtil {
     AccessTestAction checkMultiQualifierRead = new AccessTestAction() {
       @Override
       public Void run() throws Exception {
-        checkTablePerms(TEST_UTIL, TEST_TABLE.getTableName(), new Permission[] {
-          new TablePermission(TEST_TABLE.getTableName(), TEST_FAMILY, TEST_Q1,
-            Permission.Action.READ),
-          new TablePermission(TEST_TABLE.getTableName(), TEST_FAMILY, TEST_Q2,
-            Permission.Action.READ), });
+        checkTablePerms(TEST_UTIL, TEST_TABLE.getTableName(),
+          new Permission[] {
+              Permission.newBuilder(TEST_TABLE.getTableName()).withFamily(TEST_FAMILY)
+                  .withQualifier(TEST_Q1).withActions(Action.READ).build(),
+              Permission.newBuilder(TEST_TABLE.getTableName()).withFamily(TEST_FAMILY)
+                  .withQualifier(TEST_Q2).withActions(Action.READ).build() });
         return null;
       }
     };
@@ -397,11 +398,14 @@ public class TestWithDisabledAuthorization extends SecureTestUtil {
     AccessTestAction checkMultiQualifierReadWrite = new AccessTestAction() {
       @Override
       public Void run() throws Exception {
-        checkTablePerms(TEST_UTIL, TEST_TABLE.getTableName(), new Permission[] {
-            new TablePermission(TEST_TABLE.getTableName(), TEST_FAMILY, TEST_Q1,
-              Permission.Action.READ, Permission.Action.WRITE),
-            new TablePermission(TEST_TABLE.getTableName(), TEST_FAMILY, TEST_Q2,
-              Permission.Action.READ, Permission.Action.WRITE), });
+        checkTablePerms(TEST_UTIL, TEST_TABLE.getTableName(),
+          new Permission[] {
+              Permission.newBuilder(TEST_TABLE.getTableName()).withFamily(TEST_FAMILY)
+                  .withQualifier(TEST_Q1)
+                  .withActions(Permission.Action.READ, Permission.Action.WRITE).build(),
+              Permission.newBuilder(TEST_TABLE.getTableName()).withFamily(TEST_FAMILY)
+                  .withQualifier(TEST_Q2)
+                  .withActions(Permission.Action.READ, Permission.Action.WRITE).build() });
         return null;
       }
     };
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestZKPermissionWatcher.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestZKPermissionWatcher.java
index cfd6512..327c49d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestZKPermissionWatcher.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestZKPermissionWatcher.java
@@ -113,8 +113,8 @@ public class TestZKPermissionWatcher {
 
     // update ACL: george RW
     List<UserPermission> acl = new ArrayList<>(1);
-    acl.add(new UserPermission(george.getShortName(), TEST_TABLE,
-      Permission.Action.READ, Permission.Action.WRITE));
+    acl.add(new UserPermission(george.getShortName(), Permission.newBuilder(TEST_TABLE)
+        .withActions(Permission.Action.READ, Permission.Action.WRITE).build()));
     ListMultimap<String, UserPermission> multimap = ArrayListMultimap.create();
     multimap.putAll(george.getShortName(), acl);
     byte[] serialized = AccessControlLists.writePermissionsAsBytes(multimap, conf);
@@ -141,7 +141,8 @@ public class TestZKPermissionWatcher {
 
     // update ACL: hubert R
     List<UserPermission> acl2 = new ArrayList<>(1);
-    acl2.add(new UserPermission(hubert.getShortName(), TEST_TABLE, TablePermission.Action.READ));
+    acl2.add(new UserPermission(hubert.getShortName(),
+        Permission.newBuilder(TEST_TABLE).withActions(TablePermission.Action.READ).build()));
     final long mtimeA = AUTH_A.getMTime();
     multimap.putAll(hubert.getShortName(), acl2);
     byte[] serialized2 = AccessControlLists.writePermissionsAsBytes(multimap, conf);
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java
index aa0c2a8..508a623 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java
@@ -56,7 +56,7 @@ import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.replication.SyncReplicationState;
-import org.apache.hadoop.hbase.security.access.Permission;
+import org.apache.hadoop.hbase.security.access.UserPermission;
 import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
 import org.apache.hadoop.hbase.thrift2.ThriftUtilities;
 import org.apache.hadoop.hbase.thrift2.generated.TColumnFamilyDescriptor;
@@ -1101,12 +1101,12 @@ public class ThriftAdmin implements Admin {
   }
 
   @Override
-  public void grant(String userName, Permission permission, boolean mergeExistingPermissions) {
+  public void grant(UserPermission userPermission, boolean mergeExistingPermissions) {
     throw new NotImplementedException("grant not supported in ThriftAdmin");
   }
 
   @Override
-  public void revoke(String userName, Permission permission) {
+  public void revoke(UserPermission userPermission) {
     throw new NotImplementedException("revoke not supported in ThriftAdmin");
   }
 }


[hbase] 43/49: HBASE-22053 Changed zookeeper URL from http://hadoop.apache.org to https://zookeeper.apache.org Replaced version 3.1.1 with 3.3.3 as 3.1.1 is deprecated.

Posted by op...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

openinx pushed a commit to branch HBASE-21879
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit def29c28b9c9b1f1113e09cd3306943e6e9c4560
Author: subrat.mishra <su...@yahoo.com>
AuthorDate: Mon Mar 25 17:41:15 2019 +0530

    HBASE-22053 Changed zookeeper URL from http://hadoop.apache.org to https://zookeeper.apache.org Replaced version 3.1.1 with 3.3.3 as 3.1.1 is deprecated.
    
    Signed-off-by: Toshihiro Suzuki <br...@gmail.com>
---
 hbase-common/src/main/resources/hbase-default.xml | 6 +++---
 src/main/asciidoc/_chapters/hbase-default.adoc    | 6 +++---
 src/main/asciidoc/_chapters/zookeeper.adoc        | 2 +-
 3 files changed, 7 insertions(+), 7 deletions(-)

diff --git a/hbase-common/src/main/resources/hbase-default.xml b/hbase-common/src/main/resources/hbase-default.xml
index c8ab48a..b738734 100644
--- a/hbase-common/src/main/resources/hbase-default.xml
+++ b/hbase-common/src/main/resources/hbase-default.xml
@@ -374,7 +374,7 @@ possible configurations would overwhelm and obscure the important.
     <description>ZooKeeper session timeout in milliseconds. It is used in two different ways.
       First, this value is used in the ZK client that HBase uses to connect to the ensemble.
       It is also used by HBase when it starts a ZK server and it is passed as the 'maxSessionTimeout'.
-      See http://hadoop.apache.org/zookeeper/docs/current/zookeeperProgrammers.html#ch_zkSessions.
+      See https://zookeeper.apache.org/doc/current/zookeeperProgrammers.html#ch_zkSessions.
       For example, if an HBase region server connects to a ZK ensemble that's also managed
       by HBase, then the session timeout will be the one specified by this configuration.
       But, a region server that connects to an ensemble managed with a different configuration
@@ -419,14 +419,14 @@ possible configurations would overwhelm and obscure the important.
     <name>hbase.zookeeper.peerport</name>
     <value>2888</value>
     <description>Port used by ZooKeeper peers to talk to each other.
-    See http://hadoop.apache.org/zookeeper/docs/r3.1.1/zookeeperStarted.html#sc_RunningReplicatedZooKeeper
+    See https://zookeeper.apache.org/doc/r3.3.3/zookeeperStarted.html#sc_RunningReplicatedZooKeeper
     for more information.</description>
   </property>
   <property>
     <name>hbase.zookeeper.leaderport</name>
     <value>3888</value>
     <description>Port used by ZooKeeper for leader election.
-    See http://hadoop.apache.org/zookeeper/docs/r3.1.1/zookeeperStarted.html#sc_RunningReplicatedZooKeeper
+    See https://zookeeper.apache.org/doc/r3.3.3/zookeeperStarted.html#sc_RunningReplicatedZooKeeper
     for more information.</description>
   </property>
   <!-- End of properties used to generate ZooKeeper host:port quorum list. -->
diff --git a/src/main/asciidoc/_chapters/hbase-default.adoc b/src/main/asciidoc/_chapters/hbase-default.adoc
index ccd5f6f..b677714 100644
--- a/src/main/asciidoc/_chapters/hbase-default.adoc
+++ b/src/main/asciidoc/_chapters/hbase-default.adoc
@@ -459,7 +459,7 @@ The host name or IP address of the name server (DNS)
 ZooKeeper session timeout in milliseconds. It is used in two different ways.
       First, this value is used in the ZK client that HBase uses to connect to the ensemble.
       It is also used by HBase when it starts a ZK server and it is passed as the 'maxSessionTimeout'. See
-      https://hadoop.apache.org/zookeeper/docs/current/zookeeperProgrammers.html#ch_zkSessions.
+      https://zookeeper.apache.org/doc/current/zookeeperProgrammers.html#ch_zkSessions.
       For example, if an HBase region server connects to a ZK ensemble that's also managed
       by HBase, then the
       session timeout will be the one specified by this configuration. But, a region server that connects
@@ -523,7 +523,7 @@ The host name or IP address of the name server (DNS)
 +
 .Description
 Port used by ZooKeeper peers to talk to each other.
-    See https://hadoop.apache.org/zookeeper/docs/r3.1.1/zookeeperStarted.html#sc_RunningReplicatedZooKeeper
+    See https://zookeeper.apache.org/doc/r3.3.3/zookeeperStarted.html#sc_RunningReplicatedZooKeeper
     for more information.
 +
 .Default
@@ -535,7 +535,7 @@ Port used by ZooKeeper peers to talk to each other.
 +
 .Description
 Port used by ZooKeeper for leader election.
-    See https://hadoop.apache.org/zookeeper/docs/r3.1.1/zookeeperStarted.html#sc_RunningReplicatedZooKeeper
+    See https://zookeeper.apache.org/doc/r3.3.3/zookeeperStarted.html#sc_RunningReplicatedZooKeeper
     for more information.
 +
 .Default
diff --git a/src/main/asciidoc/_chapters/zookeeper.adoc b/src/main/asciidoc/_chapters/zookeeper.adoc
index 10d3417..75dd71d 100644
--- a/src/main/asciidoc/_chapters/zookeeper.adoc
+++ b/src/main/asciidoc/_chapters/zookeeper.adoc
@@ -135,7 +135,7 @@ ${HBASE_HOME}/bin/hbase-daemons.sh {start,stop} zookeeper
 Note that you can use HBase in this manner to spin up a ZooKeeper cluster, unrelated to HBase.
 Just make sure to set `HBASE_MANAGES_ZK` to `false`      if you want it to stay up across HBase restarts so that when HBase shuts down, it doesn't take ZooKeeper down with it.
 
-For more information about running a distinct ZooKeeper cluster, see the ZooKeeper link:https://hadoop.apache.org/zookeeper/docs/current/zookeeperStarted.html[Getting
+For more information about running a distinct ZooKeeper cluster, see the ZooKeeper link:https://zookeeper.apache.org/doc/current/zookeeperStarted.html[Getting
         Started Guide].
 Additionally, see the link:https://wiki.apache.org/hadoop/ZooKeeper/FAQ#A7[ZooKeeper Wiki] or the link:https://zookeeper.apache.org/doc/r3.3.3/zookeeperAdmin.html#sc_zkMulitServerSetup[ZooKeeper
         documentation] for more information on ZooKeeper sizing.


[hbase] 05/49: HBASE-22022 nightly fails rat check down in the dev-support/hbase_nightly_source-artifact.sh check

Posted by op...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

openinx pushed a commit to branch HBASE-21879
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit 26a6761424fc7c26224d1b35b006e5c621481ce3
Author: stack <st...@apache.org>
AuthorDate: Sun Mar 10 15:00:04 2019 -0700

    HBASE-22022 nightly fails rat check down in the dev-support/hbase_nightly_source-artifact.sh check
---
 dev-support/hbase_nightly_source-artifact.sh | 5 ++++-
 1 file changed, 4 insertions(+), 1 deletion(-)

diff --git a/dev-support/hbase_nightly_source-artifact.sh b/dev-support/hbase_nightly_source-artifact.sh
index c435c48..cd17a32 100755
--- a/dev-support/hbase_nightly_source-artifact.sh
+++ b/dev-support/hbase_nightly_source-artifact.sh
@@ -182,5 +182,8 @@ if mvn -DskipTests -Prelease --batch-mode -Dmaven.repo.local="${m2_tarbuild}" cl
     fi
   done
 fi
-echo "Building a binary tarball from the source tarball failed. see srtarball_install.log for details."
+echo "Building a binary tarball from the source tarball failed. see srctarball_install.log for details."
+# Copy up the rat.txt to the working dir so available in build archive in case rat complaints.
+# rat.txt can be under any module target dir... copy them all up renaming them to include parent dir as we go.
+find ${unpack_dir} -name rat.txt -type f | while IFS= read -r NAME; do cp -v "$NAME" "${working_dir}/${NAME//\//_}"; done
 exit 1


[hbase] 26/49: HBASE-21895 - Error prone upgrade

Posted by op...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

openinx pushed a commit to branch HBASE-21879
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit f337f429c39537d0835b1401ef0d9086c708e0cb
Author: Kevin Risden <kr...@apache.org>
AuthorDate: Fri Mar 15 15:04:30 2019 -0400

    HBASE-21895 - Error prone upgrade
    
    * Upgrades to error prone 2.3.3
    * Moves to error prone plugin to support 9+ JDKs
    * Removes custom error prone plugin due to no usage
    
    Signed-off-by: zhangduo <zh...@apache.org>
---
 hbase-build-configuration/pom.xml                  | 65 +++++++++-------
 hbase-build-support/hbase-error-prone/pom.xml      | 86 ----------------------
 .../hadoop/hbase/errorprone/AlwaysPasses.java      | 40 ----------
 hbase-build-support/pom.xml                        | 78 --------------------
 .../TestMultiTableSnapshotInputFormatImpl.java     |  5 ++
 .../hadoop/hbase/procedure2/util/DelayedUtil.java  |  5 ++
 hbase-protocol-shaded/pom.xml                      |  2 -
 .../hbase/client/TestMultipleTimestamps.java       |  1 -
 pom.xml                                            |  9 +--
 9 files changed, 48 insertions(+), 243 deletions(-)

diff --git a/hbase-build-configuration/pom.xml b/hbase-build-configuration/pom.xml
index a617083..0440227 100644
--- a/hbase-build-configuration/pom.xml
+++ b/hbase-build-configuration/pom.xml
@@ -68,6 +68,31 @@
       <activation>
         <activeByDefault>false</activeByDefault>
       </activation>
+      <properties>
+        <!-- https://errorprone.info/docs/installation Maven section has details -->
+        <!-- required when compiling with JDK 8 -->
+        <javac.version>9+181-r4173-1</javac.version>
+      </properties>
+      <dependencies>
+        <dependency>
+          <groupId>com.google.errorprone</groupId>
+          <artifactId>error_prone_core</artifactId>
+          <version>${error-prone.version}</version>
+          <scope>provided</scope>
+          <exclusions>
+            <exclusion>
+              <groupId>com.google.code.findbugs</groupId>
+              <artifactId>jsr305</artifactId>
+            </exclusion>
+          </exclusions>
+        </dependency>
+        <dependency>
+          <groupId>com.google.errorprone</groupId>
+          <artifactId>javac</artifactId>
+          <version>${javac.version}</version>
+          <scope>provided</scope>
+        </dependency>
+      </dependencies>
       <build>
         <plugins>
           <!-- Turn on error-prone -->
@@ -76,42 +101,26 @@
             <artifactId>maven-compiler-plugin</artifactId>
             <version>${maven.compiler.version}</version>
             <configuration>
-              <compilerId>javac-with-errorprone</compilerId>
-              <forceJavacCompilerUse>true</forceJavacCompilerUse>
+              <source>${compileSource}</source>
+              <target>${compileSource}</target>
+              <!-- required when compiling with JDK 8 -->
+              <fork>true</fork>
               <showWarnings>true</showWarnings>
               <compilerArgs>
-                <arg>-XepDisableWarningsInGeneratedCode</arg>
-                <arg>-Xep:FallThrough:OFF</arg> <!-- already in findbugs -->
-                <arg>-Xep:ClassNewInstance:ERROR</arg>
-                <arg>-Xep:MissingDefault:ERROR</arg>
+                <arg>-XDcompilePolicy=simple</arg>
+                <!-- All -Xep need to be on single line see: https://github.com/google/error-prone/pull/1115 -->
+                <arg>-Xplugin:ErrorProne -XepDisableWarningsInGeneratedCode -Xep:FallThrough:OFF -Xep:ClassNewInstance:ERROR -Xep:MissingDefault:ERROR</arg>
+                <!-- Required when compiling with JDK 8 -->
+                <arg>-J-Xbootclasspath/p:${settings.localRepository}/com/google/errorprone/javac/${javac.version}/javac-${javac.version}.jar</arg>
               </compilerArgs>
               <annotationProcessorPaths>
                 <path>
-                  <groupId>org.apache.hbase</groupId>
-                  <artifactId>hbase-error-prone</artifactId>
-                  <version>${project.version}</version>
+                  <groupId>com.google.errorprone</groupId>
+                  <artifactId>error_prone_core</artifactId>
+                  <version>${error-prone.version}</version>
                 </path>
               </annotationProcessorPaths>
             </configuration>
-            <dependencies>
-              <dependency>
-                <groupId>org.codehaus.plexus</groupId>
-                <artifactId>plexus-compiler-javac-errorprone</artifactId>
-                <version>${plexus.errorprone.javac.version}</version>
-              </dependency>
-              <!-- override plexus-compiler-javac-errorprone's dependency on
-                Error Prone with the latest version -->
-              <dependency>
-                <groupId>com.google.errorprone</groupId>
-                <artifactId>error_prone_core</artifactId>
-                <version>${error-prone.version}</version>
-              </dependency>
-              <dependency>
-                <groupId>org.apache.hbase</groupId>
-                <artifactId>hbase-error-prone</artifactId>
-                <version>${project.version}</version>
-              </dependency>
-            </dependencies>
           </plugin>
         </plugins>
       </build>
diff --git a/hbase-build-support/hbase-error-prone/pom.xml b/hbase-build-support/hbase-error-prone/pom.xml
deleted file mode 100644
index 161ed24..0000000
--- a/hbase-build-support/hbase-error-prone/pom.xml
+++ /dev/null
@@ -1,86 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
-<!--
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
--->
-  <modelVersion>4.0.0</modelVersion>
-  <parent>
-    <artifactId>hbase-build-support</artifactId>
-    <groupId>org.apache.hbase</groupId>
-    <version>3.0.0-SNAPSHOT</version>
-    <relativePath>..</relativePath>
-  </parent>
-  <artifactId>hbase-error-prone</artifactId>
-  <version>3.0.0-SNAPSHOT</version>
-  <name>Apache HBase - Error Prone Rules</name>
-  <description>Module to hold error prone custom rules for HBase.</description>
-
-  <dependencies>
-    <dependency>
-      <groupId>com.google.errorprone</groupId>
-      <artifactId>error_prone_annotation</artifactId>
-      <version>${error-prone.version}</version>
-      <scope>provided</scope>
-      <exclusions>
-        <exclusion>
-          <groupId>com.google.guava</groupId>
-          <artifactId>guava</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <dependency>
-      <!--mvn dependency:analyze says this is not used but compile fails
-            without it; going w/ the compiler's view of the world-->
-      <groupId>com.google.auto.service</groupId>
-      <artifactId>auto-service</artifactId>
-      <version>1.0-rc3</version>
-      <optional>true</optional>
-    </dependency>
-    <dependency>
-      <groupId>com.google.errorprone</groupId>
-      <artifactId>error_prone_check_api</artifactId>
-      <version>${error-prone.version}</version>
-      <scope>provided</scope>
-      <exclusions>
-        <exclusion>
-          <groupId>com.google.code.findbugs</groupId>
-          <artifactId>jsr305</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <dependency>
-      <groupId>com.google.errorprone</groupId>
-      <artifactId>javac</artifactId>
-      <version>9-dev-r4023-3</version>
-      <scope>provided</scope>
-    </dependency>
-  </dependencies>
-
-  <build>
-    <plugins>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-checkstyle-plugin</artifactId>
-        <configuration>
-          <failOnViolation>true</failOnViolation>
-        </configuration>
-      </plugin>
-    </plugins>
-  </build>
-</project>
\ No newline at end of file
diff --git a/hbase-build-support/hbase-error-prone/src/main/java/org/apache/hadoop/hbase/errorprone/AlwaysPasses.java b/hbase-build-support/hbase-error-prone/src/main/java/org/apache/hadoop/hbase/errorprone/AlwaysPasses.java
deleted file mode 100644
index 9962835..0000000
--- a/hbase-build-support/hbase-error-prone/src/main/java/org/apache/hadoop/hbase/errorprone/AlwaysPasses.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.errorprone;
-
-import com.google.auto.service.AutoService;
-import com.google.errorprone.BugPattern;
-import com.google.errorprone.VisitorState;
-import com.google.errorprone.bugpatterns.BugChecker;
-import com.google.errorprone.matchers.Description;
-import com.sun.source.tree.CompilationUnitTree;
-
-@AutoService(BugChecker.class)
-@BugPattern(name = "AlwaysPasses",
-    category = BugPattern.Category.JDK,
-    summary = "A placeholder rule that never matches.",
-    severity = BugPattern.SeverityLevel.ERROR,
-    suppressionAnnotations = {},
-    linkType = BugPattern.LinkType.NONE)
-public class AlwaysPasses extends BugChecker implements BugChecker.CompilationUnitTreeMatcher {
-  @Override
-  public Description matchCompilationUnit(CompilationUnitTree tree, VisitorState state) {
-    return Description.NO_MATCH;
-  }
-}
diff --git a/hbase-build-support/pom.xml b/hbase-build-support/pom.xml
deleted file mode 100644
index f13c3f1..0000000
--- a/hbase-build-support/pom.xml
+++ /dev/null
@@ -1,78 +0,0 @@
-<?xml version="1.0"?>
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-  <!--
-  /**
-   * Licensed to the Apache Software Foundation (ASF) under one
-   * or more contributor license agreements.  See the NOTICE file
-   * distributed with this work for additional information
-   * regarding copyright ownership.  The ASF licenses this file
-   * to you under the Apache License, Version 2.0 (the
-   * "License"); you may not use this file except in compliance
-   * with the License.  You may obtain a copy of the License at
-   *
-   *     http://www.apache.org/licenses/LICENSE-2.0
-   *
-   * Unless required by applicable law or agreed to in writing, software
-   * distributed under the License is distributed on an "AS IS" BASIS,
-   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-   * See the License for the specific language governing permissions and
-   * limitations under the License.
-   */
-  -->
-  <modelVersion>4.0.0</modelVersion>
-  <parent>
-    <artifactId>hbase</artifactId>
-    <groupId>org.apache.hbase</groupId>
-    <version>3.0.0-SNAPSHOT</version>
-    <relativePath>..</relativePath>
-  </parent>
-
-  <artifactId>hbase-build-support</artifactId>
-  <name>Apache HBase - Build Support</name>
-  <description>Parent module for build-support artifacts</description>
-
-  <packaging>pom</packaging>
-  <properties>
-    <!-- Don't make a test-jar -->
-    <maven.test.skip>true</maven.test.skip>
-    <!-- Don't make a source-jar -->
-    <source.skip>true</source.skip>
-    <!-- Don't make a site -->
-    <maven.site.skip>true</maven.site.skip>
-  </properties>
-  <modules>
-    <module>hbase-error-prone</module>
-  </modules>
-  <build>
-    <pluginManagement>
-      <plugins>
-        <!-- This entry overrides the excludeFileFilter element in the findbugs
-             configuration of the hbase/pom.xml file. This override specifies that
-             the excluded-filter-file is found TWO levels up from a grandchild project. -->
-        <plugin>
-          <groupId>org.codehaus.mojo</groupId>
-          <artifactId>findbugs-maven-plugin</artifactId>
-          <configuration>
-            <excludeFilterFile>${project.basedir}/../../dev-support/findbugs-exclude.xml</excludeFilterFile>
-          </configuration>
-        </plugin>
-        <plugin>
-          <!--Make it so assembly:single does nothing in here-->
-          <artifactId>maven-assembly-plugin</artifactId>
-          <configuration>
-            <skipAssembly>true</skipAssembly>
-          </configuration>
-        </plugin>
-      </plugins>
-    </pluginManagement>
-    <plugins>
-      <plugin>
-        <!--Make it so assembly:single does nothing in here-->
-        <artifactId>maven-assembly-plugin</artifactId>
-        <configuration>
-          <skipAssembly>true</skipAssembly>
-        </configuration>
-      </plugin>
-    </plugins>
-  </build>
-</project>
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableSnapshotInputFormatImpl.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableSnapshotInputFormatImpl.java
index 685e519..9fec1b5 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableSnapshotInputFormatImpl.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableSnapshotInputFormatImpl.java
@@ -134,6 +134,11 @@ public class TestMultiTableSnapshotInputFormatImpl {
     }
 
     @Override
+    public int hashCode() {
+      return Objects.hash(startRow, stopRow);
+    }
+
+    @Override
     public String toString() {
       return org.apache.hbase.thirdparty.com.google.common.base.MoreObjects.
         toStringHelper(this).add("startRow", startRow)
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/util/DelayedUtil.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/util/DelayedUtil.java
index 009f421..471e056 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/util/DelayedUtil.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/util/DelayedUtil.java
@@ -65,6 +65,11 @@ public final class DelayedUtil {
     }
 
     @Override
+    public int hashCode() {
+      return Objects.hash(this);
+    }
+
+    @Override
     public String toString() {
       return getClass().getSimpleName() + "(POISON)";
     }
diff --git a/hbase-protocol-shaded/pom.xml b/hbase-protocol-shaded/pom.xml
index f5a1cab..2d4a7b3 100644
--- a/hbase-protocol-shaded/pom.xml
+++ b/hbase-protocol-shaded/pom.xml
@@ -169,10 +169,8 @@
                   <exclude>org.apache.hbase.thirdparty:*</exclude>
                   <exclude>com.google.protobuf:protobuf-java</exclude>
                   <exclude>com.google.code.findbugs:*</exclude>
-                  <exclude>com.google.errorprone:error_prone_annotations</exclude>
                   <exclude>com.google.j2objc:j2objc-annotations</exclude>
                   <exclude>org.codehaus.mojo:animal-sniffer-annotations</exclude>
-                  <exclude>org.codehaus.mojo:animal-sniffer-annotations</exclude>
                   <exclude>junit:junit</exclude>
                   <exclude>log4j:log4j</exclude>
                   <exclude>commons-logging:commons-logging</exclude>
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultipleTimestamps.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultipleTimestamps.java
index ac6b9d1..116aa9b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultipleTimestamps.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultipleTimestamps.java
@@ -448,7 +448,6 @@ public class TestMultipleTimestamps {
       Integer[] rowIndexes, Integer[] columnIndexes,
       Long[] versions, int maxVersions)
   throws IOException {
-    Arrays.asList(rowIndexes);
     byte startRow[] = Bytes.toBytes("row:" +
         Collections.min( Arrays.asList(rowIndexes)));
     byte endRow[] = Bytes.toBytes("row:" +
diff --git a/pom.xml b/pom.xml
index f408fc4..d3b9fd3 100755
--- a/pom.xml
+++ b/pom.xml
@@ -61,7 +61,6 @@
   </licenses>
 
   <modules>
-    <module>hbase-build-support</module>
     <module>hbase-build-configuration</module>
     <module>hbase-replication</module>
     <module>hbase-mapreduce</module>
@@ -1568,7 +1567,7 @@
     <buildnumber.maven.version>1.4</buildnumber.maven.version>
     <checkstyle.version>8.11</checkstyle.version>
     <exec.maven.version>1.6.0</exec.maven.version>
-    <error-prone.version>2.2.0</error-prone.version>
+    <error-prone.version>2.3.3</error-prone.version>
     <findbugs-annotations>1.3.9-1</findbugs-annotations>
     <findbugs.maven.version>3.0.4</findbugs.maven.version>
     <jamon.plugin.version>2.4.2</jamon.plugin.version>
@@ -1580,7 +1579,6 @@
     <maven.eclipse.version>2.10</maven.eclipse.version>
     <maven.warbucks.version>1.1.0</maven.warbucks.version>
     <os.maven.version>1.5.0.Final</os.maven.version>
-    <plexus.errorprone.javac.version>2.8.2</plexus.errorprone.javac.version>
     <spotbugs.version>3.1.11</spotbugs.version>
     <wagon.ssh.version>2.12</wagon.ssh.version>
     <xml.maven.version>1.0.1</xml.maven.version>
@@ -1666,11 +1664,6 @@
       </dependency>
       <dependency>
         <groupId>org.apache.hbase</groupId>
-        <artifactId>hbase-error-prone</artifactId>
-        <version>${project.version}</version>
-      </dependency>
-      <dependency>
-        <groupId>org.apache.hbase</groupId>
         <artifactId>hbase-common</artifactId>
         <version>${project.version}</version>
       </dependency>


[hbase] 42/49: HBASE-22097 Modify the description of split command in shell

Posted by op...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

openinx pushed a commit to branch HBASE-21879
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit 03db5fea7d021e7c4b03e086970de43a5b05be9d
Author: Toshihiro Suzuki <br...@gmail.com>
AuthorDate: Sun Mar 24 00:56:10 2019 +0900

    HBASE-22097 Modify the description of split command in shell
    
    Signed-off-by: Peter Somogyi <ps...@apache.org>
    Signed-off-by: Xu Cang <xu...@apache.org>
---
 hbase-shell/src/main/ruby/shell/commands/split.rb | 11 ++++++-----
 1 file changed, 6 insertions(+), 5 deletions(-)

diff --git a/hbase-shell/src/main/ruby/shell/commands/split.rb b/hbase-shell/src/main/ruby/shell/commands/split.rb
index c7a1e29..ede6987 100644
--- a/hbase-shell/src/main/ruby/shell/commands/split.rb
+++ b/hbase-shell/src/main/ruby/shell/commands/split.rb
@@ -25,11 +25,12 @@ module Shell
 Split entire table or pass a region to split individual region.  With the
 second parameter, you can specify an explicit split key for the region.
 Examples:
-    split 'tableName'
-    split 'namespace:tableName'
-    split 'regionName' # format: 'tableName,startKey,id'
-    split 'tableName', 'splitKey'
-    split 'regionName', 'splitKey'
+    split 'TABLENAME'
+    split 'REGIONNAME'
+    split 'ENCODED_REGIONNAME'
+    split 'TABLENAME', 'splitKey'
+    split 'REGIONNAME', 'splitKey'
+    split 'ENCODED_REGIONNAME', 'splitKey'
 EOF
       end
 


[hbase] 01/49: HBASE-21810 bulkload support set hfile compression on client

Posted by op...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

openinx pushed a commit to branch HBASE-21879
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit 0da5b263833794e814a1e1a8ff98a586e34a01e3
Author: chenyechao <ch...@gmail.com>
AuthorDate: Mon Mar 11 19:28:08 2019 +0800

    HBASE-21810 bulkload support set hfile compression on client
    
    Signed-off-by: Guanghao Zhang <zg...@apache.org>
---
 .../hadoop/hbase/mapreduce/HFileOutputFormat2.java | 15 +++++++-
 .../hbase/mapreduce/TestHFileOutputFormat2.java    | 43 ++++++++++++++++++++++
 2 files changed, 56 insertions(+), 2 deletions(-)

diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java
index bac3618..1e96232 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java
@@ -160,9 +160,11 @@ public class HFileOutputFormat2
   // This constant is public since the client can modify this when setting
   // up their conf object and thus refer to this symbol.
   // It is present for backwards compatibility reasons. Use it only to
-  // override the auto-detection of datablock encoding.
+  // override the auto-detection of datablock encoding and compression.
   public static final String DATABLOCK_ENCODING_OVERRIDE_CONF_KEY =
       "hbase.mapreduce.hfileoutputformat.datablock.encoding";
+  public static final String COMPRESSION_OVERRIDE_CONF_KEY =
+      "hbase.mapreduce.hfileoutputformat.compression";
 
   /**
    * Keep locality while generating HFiles for bulkload. See HBASE-12596
@@ -210,6 +212,14 @@ public class HFileOutputFormat2
         Compression.Algorithm.NONE.getName());
     final Algorithm defaultCompression = HFileWriterImpl
         .compressionByName(defaultCompressionStr);
+    String compressionStr = conf.get(COMPRESSION_OVERRIDE_CONF_KEY);
+    final Algorithm overriddenCompression;
+    if (compressionStr != null) {
+      overriddenCompression = Compression.getCompressionAlgorithmByName(compressionStr);
+    } else {
+      overriddenCompression = null;
+    }
+
     final boolean compactionExclude = conf.getBoolean(
         "hbase.mapreduce.hfileoutputformat.compaction.exclude", false);
 
@@ -399,7 +409,8 @@ public class HFileOutputFormat2
                   new Path(getTableRelativePath(tableName), Bytes.toString(family)));
         }
         WriterLength wl = new WriterLength();
-        Algorithm compression = compressionMap.get(tableAndFamily);
+        Algorithm compression = overriddenCompression;
+        compression = compression == null ? compressionMap.get(tableAndFamily) : compression;
         compression = compression == null ? defaultCompression : compression;
         BloomType bloomType = bloomTypeMap.get(tableAndFamily);
         bloomType = bloomType == null ? BloomType.NONE : bloomType;
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java
index cbf5ff8..e8bb805 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java
@@ -1551,5 +1551,48 @@ public class TestHFileOutputFormat2  {
     Assert.assertTrue(partitionPathString.startsWith(fooHomeDirectory.toString()));
     Assert.assertTrue(fs.exists(new Path(partitionPathString)));
   }
+
+  @Test
+  public void TestConfigureCompression() throws Exception {
+    Configuration conf = new Configuration(this.util.getConfiguration());
+    RecordWriter<ImmutableBytesWritable, Cell> writer = null;
+    TaskAttemptContext context = null;
+    Path dir = util.getDataTestDir("TestConfigureCompression");
+    String hfileoutputformatCompression = "gz";
+
+    try {
+      conf.set(HFileOutputFormat2.OUTPUT_TABLE_NAME_CONF_KEY, TABLE_NAMES[0].getNameAsString());
+      conf.setBoolean(HFileOutputFormat2.LOCALITY_SENSITIVE_CONF_KEY, false);
+
+      conf.set(HFileOutputFormat2.COMPRESSION_OVERRIDE_CONF_KEY, hfileoutputformatCompression);
+
+      Job job = Job.getInstance(conf);
+      FileOutputFormat.setOutputPath(job, dir);
+      context = createTestTaskAttemptContext(job);
+      HFileOutputFormat2 hof = new HFileOutputFormat2();
+      writer = hof.getRecordWriter(context);
+      final byte[] b = Bytes.toBytes("b");
+
+      KeyValue kv = new KeyValue(b, b, b, HConstants.LATEST_TIMESTAMP, b);
+      writer.write(new ImmutableBytesWritable(), kv);
+      writer.close(context);
+      writer = null;
+      FileSystem fs = dir.getFileSystem(conf);
+      RemoteIterator<LocatedFileStatus> iterator = fs.listFiles(dir, true);
+      while (iterator.hasNext()) {
+        LocatedFileStatus keyFileStatus = iterator.next();
+        HFile.Reader reader =
+            HFile.createReader(fs, keyFileStatus.getPath(), new CacheConfig(conf), true, conf);
+        assertEquals(reader.getCompressionAlgorithm().getName(), hfileoutputformatCompression);
+      }
+    } finally {
+      if (writer != null && context != null) {
+        writer.close(context);
+      }
+      dir.getFileSystem(conf).delete(dir, true);
+    }
+
+  }
+
 }
 


[hbase] 39/49: HBASE-21911 Move getUserPermissions from regionserver to master

Posted by op...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

openinx pushed a commit to branch HBASE-21879
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit 81ee1dbaa1ad49ae7532bf07199ad6941f3581fd
Author: meiyi <my...@gamil.com>
AuthorDate: Mon Mar 25 10:35:27 2019 +0800

    HBASE-21911 Move getUserPermissions from regionserver to master
    
    Signed-off-by: Guanghao Zhang <zg...@apache.org>
---
 .../java/org/apache/hadoop/hbase/client/Admin.java |  11 ++
 .../org/apache/hadoop/hbase/client/AsyncAdmin.java |  10 ++
 .../hadoop/hbase/client/AsyncHBaseAdmin.java       |   7 ++
 .../hbase/client/ConnectionImplementation.java     |   8 ++
 .../org/apache/hadoop/hbase/client/HBaseAdmin.java |  20 +++
 .../hadoop/hbase/client/RawAsyncHBaseAdmin.java    |  17 +++
 .../hbase/client/ShortCircuitMasterConnection.java |   8 ++
 .../hbase/security/access/AccessControlClient.java |  67 ++++------
 .../hbase/security/access/AccessControlUtil.java   |  12 ++
 .../security/access/GetUserPermissionsRequest.java | 136 +++++++++++++++++++++
 .../security/access/ShadedAccessControlUtil.java   |  38 ++++++
 .../src/main/protobuf/AccessControl.proto          |   3 +
 .../src/main/protobuf/Master.proto                 |   2 +
 .../hadoop/hbase/coprocessor/MasterObserver.java   |  30 +++++
 .../hadoop/hbase/master/MasterCoprocessorHost.java |  20 +++
 .../hadoop/hbase/master/MasterRpcServices.java     |  57 +++++++++
 .../hbase/security/access/AccessControlLists.java  |  13 +-
 .../hbase/security/access/AccessController.java    | 100 ++++++---------
 .../client/TestAsyncAccessControlAdminApi.java     |  74 +++++++++++
 .../hbase/security/access/SecureTestUtil.java      |   6 +-
 .../security/access/TestAccessController.java      | 124 +++++--------------
 .../security/access/TestNamespaceCommands.java     |  15 ++-
 .../hadoop/hbase/thrift2/client/ThriftAdmin.java   |   7 ++
 23 files changed, 570 insertions(+), 215 deletions(-)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
index b27e93d..33b44c3 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
@@ -53,6 +53,7 @@ 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.replication.SyncReplicationState;
+import org.apache.hadoop.hbase.security.access.GetUserPermissionsRequest;
 import org.apache.hadoop.hbase.security.access.UserPermission;
 import org.apache.hadoop.hbase.snapshot.HBaseSnapshotException;
 import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
@@ -2044,4 +2045,14 @@ public interface Admin extends Abortable, Closeable {
    * @throws IOException if a remote or network exception occurs
    */
   void revoke(UserPermission userPermission) throws IOException;
+
+  /**
+   * Get the global/namespace/table permissions for user
+   * @param getUserPermissionsRequest A request contains which user, global, namespace or table
+   *          permissions needed
+   * @return The user and permission list
+   * @throws IOException if a remote or network exception occurs
+   */
+  List<UserPermission> getUserPermissions(GetUserPermissionsRequest getUserPermissionsRequest)
+      throws IOException;
 }
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 f7adc16..3227f22 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
@@ -46,6 +46,7 @@ import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshotView;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.replication.SyncReplicationState;
+import org.apache.hadoop.hbase.security.access.GetUserPermissionsRequest;
 import org.apache.hadoop.hbase.security.access.UserPermission;
 import org.apache.yetus.audience.InterfaceAudience;
 
@@ -1447,4 +1448,13 @@ public interface AsyncAdmin {
    * @param userPermission user name and the specific permission
    */
   CompletableFuture<Void> revoke(UserPermission userPermission);
+
+  /**
+   * Get the global/namespace/table permissions for user
+   * @param getUserPermissionsRequest A request contains which user, global, namespace or table
+   *          permissions needed
+   * @return The user and permission list
+   */
+  CompletableFuture<List<UserPermission>>
+      getUserPermissions(GetUserPermissionsRequest getUserPermissionsRequest);
 }
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 0eceaad..07b4311 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
@@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.replication.SyncReplicationState;
+import org.apache.hadoop.hbase.security.access.GetUserPermissionsRequest;
 import org.apache.hadoop.hbase.security.access.UserPermission;
 import org.apache.hadoop.hbase.util.FutureUtils;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -812,4 +813,10 @@ class AsyncHBaseAdmin implements AsyncAdmin {
   public CompletableFuture<Void> revoke(UserPermission userPermission) {
     return wrap(rawAdmin.revoke(userPermission));
   }
+
+  @Override
+  public CompletableFuture<List<UserPermission>>
+      getUserPermissions(GetUserPermissionsRequest getUserPermissionsRequest) {
+    return wrap(rawAdmin.getUserPermissions(getUserPermissionsRequest));
+  }
 }
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 ff2ff2f..6bcb499 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
@@ -93,6 +93,8 @@ import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.GetUserPermissionsRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.GetUserPermissionsResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService.BlockingInterface;
@@ -1795,6 +1797,12 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
           AccessControlProtos.RevokeRequest request) throws ServiceException {
         return stub.revoke(controller, request);
       }
+
+      @Override
+      public GetUserPermissionsResponse getUserPermissions(RpcController controller,
+          GetUserPermissionsRequest request) throws ServiceException {
+        return stub.getUserPermissions(controller, request);
+      }
     };
   }
 
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 91f9584..442fd81 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
@@ -85,6 +85,7 @@ import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.replication.SyncReplicationState;
+import org.apache.hadoop.hbase.security.access.GetUserPermissionsRequest;
 import org.apache.hadoop.hbase.security.access.ShadedAccessControlUtil;
 import org.apache.hadoop.hbase.security.access.UserPermission;
 import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
@@ -110,6 +111,7 @@ import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
 
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.GrantRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.RevokeRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
@@ -3839,6 +3841,24 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
+  public List<UserPermission>
+      getUserPermissions(GetUserPermissionsRequest getUserPermissionsRequest) throws IOException {
+    return executeCallable(
+      new MasterCallable<List<UserPermission>>(getConnection(), getRpcControllerFactory()) {
+        @Override
+        protected List<UserPermission> rpcCall() throws Exception {
+          AccessControlProtos.GetUserPermissionsRequest req =
+              ShadedAccessControlUtil.buildGetUserPermissionsRequest(getUserPermissionsRequest);
+          AccessControlProtos.GetUserPermissionsResponse response =
+              this.master.getUserPermissions(getRpcController(), req);
+          return response.getUserPermissionList().stream()
+              .map(userPermission -> ShadedAccessControlUtil.toUserPermission(userPermission))
+              .collect(Collectors.toList());
+        }
+      });
+  }
+
+  @Override
   public void close() {
   }
 }
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 587c6e2..085fcf7 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
@@ -85,6 +85,7 @@ 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.replication.SyncReplicationState;
+import org.apache.hadoop.hbase.security.access.GetUserPermissionsRequest;
 import org.apache.hadoop.hbase.security.access.ShadedAccessControlUtil;
 import org.apache.hadoop.hbase.security.access.UserPermission;
 import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
@@ -106,6 +107,8 @@ import org.apache.hbase.thirdparty.io.netty.util.TimerTask;
 
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.GetUserPermissionsResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.GrantRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.GrantResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.RevokeRequest;
@@ -3782,4 +3785,18 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
           (s, c, req, done) -> s.revoke(c, req, done), resp -> null))
         .call();
   }
+
+  @Override
+  public CompletableFuture<List<UserPermission>>
+      getUserPermissions(GetUserPermissionsRequest getUserPermissionsRequest) {
+    return this.<List<UserPermission>> newMasterCaller().action((controller,
+        stub) -> this.<AccessControlProtos.GetUserPermissionsRequest, GetUserPermissionsResponse,
+            List<UserPermission>> call(controller, stub,
+              ShadedAccessControlUtil.buildGetUserPermissionsRequest(getUserPermissionsRequest),
+              (s, c, req, done) -> s.getUserPermissions(c, req, done),
+              resp -> resp.getUserPermissionList().stream()
+                .map(uPerm -> ShadedAccessControlUtil.toUserPermission(uPerm))
+                .collect(Collectors.toList())))
+        .call();
+  }
 }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java
index 090bcf9..6e80a54 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java
@@ -21,6 +21,8 @@ import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
 import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
 
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.GetUserPermissionsRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.GetUserPermissionsResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.GrantRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.GrantResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.RevokeRequest;
@@ -688,4 +690,10 @@ public class ShortCircuitMasterConnection implements MasterKeepAliveConnection {
       throws ServiceException {
     return stub.revoke(controller, request);
   }
+
+  @Override
+  public GetUserPermissionsResponse getUserPermissions(RpcController controller,
+      GetUserPermissionsRequest request) throws ServiceException {
+    return stub.getUserPermissions(controller, request);
+  }
 }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlClient.java
index c467dfb..046761e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlClient.java
@@ -249,36 +249,27 @@ public class AccessControlClient {
    */
   public static List<UserPermission> getUserPermissions(Connection connection, String tableRegex,
       String userName) throws Throwable {
-    /**
-     * TODO: Pass an rpcController HBaseRpcController controller = ((ClusterConnection)
-     * connection).getRpcControllerFactory().newController();
-     */
     List<UserPermission> permList = new ArrayList<>();
-    try (Table table = connection.getTable(ACL_TABLE_NAME)) {
-      try (Admin admin = connection.getAdmin()) {
-        CoprocessorRpcChannel service = table.coprocessorService(HConstants.EMPTY_START_ROW);
-        BlockingInterface protocol =
-            AccessControlProtos.AccessControlService.newBlockingStub(service);
-        List<TableDescriptor> htds = null;
-        if (tableRegex == null || tableRegex.isEmpty()) {
-          permList = AccessControlUtil.getUserPermissions(null, protocol, userName);
-        } else if (tableRegex.charAt(0) == '@') { // Namespaces
-          String namespaceRegex = tableRegex.substring(1);
-          for (NamespaceDescriptor nsds : admin.listNamespaceDescriptors()) { // Read out all
-                                                                              // namespaces
-            String namespace = nsds.getName();
-            if (namespace.matches(namespaceRegex)) { // Match the given namespace regex?
-              permList.addAll(AccessControlUtil.getUserPermissions(null, protocol,
-                Bytes.toBytes(namespace), userName));
-            }
-          }
-        } else { // Tables
-          htds = admin.listTableDescriptors(Pattern.compile(tableRegex), true);
-          for (TableDescriptor htd : htds) {
-            permList.addAll(AccessControlUtil.getUserPermissions(null, protocol, htd.getTableName(),
-              null, null, userName));
+    try (Admin admin = connection.getAdmin()) {
+      if (tableRegex == null || tableRegex.isEmpty()) {
+        permList = admin.getUserPermissions(
+          GetUserPermissionsRequest.newBuilder().withUserName(userName).build());
+      } else if (tableRegex.charAt(0) == '@') { // Namespaces
+        String namespaceRegex = tableRegex.substring(1);
+        for (NamespaceDescriptor nsds : admin.listNamespaceDescriptors()) { // Read out all
+                                                                            // namespaces
+          String namespace = nsds.getName();
+          if (namespace.matches(namespaceRegex)) { // Match the given namespace regex?
+            permList.addAll(admin.getUserPermissions(
+              GetUserPermissionsRequest.newBuilder(namespace).withUserName(userName).build()));
           }
         }
+      } else { // Tables
+        List<TableDescriptor> htds = admin.listTableDescriptors(Pattern.compile(tableRegex), true);
+        for (TableDescriptor htd : htds) {
+          permList.addAll(admin.getUserPermissions(GetUserPermissionsRequest
+              .newBuilder(htd.getTableName()).withUserName(userName).build()));
+        }
       }
     }
     return permList;
@@ -347,22 +338,14 @@ public class AccessControlClient {
     if (tableRegex == null || tableRegex.isEmpty() || tableRegex.charAt(0) == '@') {
       throw new IllegalArgumentException("Table name can't be null or empty or a namespace.");
     }
-    /**
-     * TODO: Pass an rpcController HBaseRpcController controller = ((ClusterConnection)
-     * connection).getRpcControllerFactory().newController();
-     */
     List<UserPermission> permList = new ArrayList<UserPermission>();
-    try (Table table = connection.getTable(ACL_TABLE_NAME)) {
-      try (Admin admin = connection.getAdmin()) {
-        CoprocessorRpcChannel service = table.coprocessorService(HConstants.EMPTY_START_ROW);
-        BlockingInterface protocol =
-            AccessControlProtos.AccessControlService.newBlockingStub(service);
-        List<TableDescriptor> htds = admin.listTableDescriptors(Pattern.compile(tableRegex), true);
-        // Retrieve table permissions
-        for (TableDescriptor htd : htds) {
-          permList.addAll(AccessControlUtil.getUserPermissions(null, protocol, htd.getTableName(),
-            columnFamily, columnQualifier, userName));
-        }
+    try (Admin admin = connection.getAdmin()) {
+      List<TableDescriptor> htds = admin.listTableDescriptors(Pattern.compile(tableRegex), true);
+      // Retrieve table permissions
+      for (TableDescriptor htd : htds) {
+        permList.addAll(admin.getUserPermissions(
+          GetUserPermissionsRequest.newBuilder(htd.getTableName()).withFamily(columnFamily)
+              .withQualifier(columnQualifier).withUserName(userName).build()));
       }
     }
     return permList;
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlUtil.java
index 05f173e..0220d89 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlUtil.java
@@ -654,7 +654,9 @@ public class AccessControlUtil {
    * @param controller RpcController
    * @param protocol the AccessControlService protocol proxy
    * @throws ServiceException on failure
+   * @deprecated Use {@link Admin#getUserPermissions(GetUserPermissionsRequest)} instead.
    */
+  @Deprecated
   public static List<UserPermission> getUserPermissions(RpcController controller,
       AccessControlService.BlockingInterface protocol) throws ServiceException {
     return getUserPermissions(controller, protocol, HConstants.EMPTY_STRING);
@@ -666,7 +668,9 @@ public class AccessControlUtil {
    * @param protocol the AccessControlService protocol proxy
    * @param userName User name, if empty then all user permissions will be retrieved.
    * @throws ServiceException
+   * @deprecated Use {@link Admin#getUserPermissions(GetUserPermissionsRequest)} instead.
    */
+  @Deprecated
   public static List<UserPermission> getUserPermissions(RpcController controller,
       AccessControlService.BlockingInterface protocol, String userName) throws ServiceException {
     AccessControlProtos.GetUserPermissionsRequest.Builder builder =
@@ -695,7 +699,9 @@ public class AccessControlUtil {
    * @param protocol the AccessControlService protocol proxy
    * @param t optional table name
    * @throws ServiceException
+   * @deprecated Use {@link Admin#getUserPermissions(GetUserPermissionsRequest)} instead.
    */
+  @Deprecated
   public static List<UserPermission> getUserPermissions(RpcController controller,
       AccessControlService.BlockingInterface protocol,
       TableName t) throws ServiceException {
@@ -712,7 +718,9 @@ public class AccessControlUtil {
    * @param columnQualifier Column qualifier
    * @param userName User name, if empty then all user permissions will be retrieved.
    * @throws ServiceException
+   * @deprecated Use {@link Admin#getUserPermissions(GetUserPermissionsRequest)} instead.
    */
+  @Deprecated
   public static List<UserPermission> getUserPermissions(RpcController controller,
       AccessControlService.BlockingInterface protocol, TableName t, byte[] columnFamily,
       byte[] columnQualifier, String userName) throws ServiceException {
@@ -751,7 +759,9 @@ public class AccessControlUtil {
    * @param protocol the AccessControlService protocol proxy
    * @param namespace name of the namespace
    * @throws ServiceException
+   * @deprecated Use {@link Admin#getUserPermissions(GetUserPermissionsRequest)} instead.
    */
+  @Deprecated
   public static List<UserPermission> getUserPermissions(RpcController controller,
       AccessControlService.BlockingInterface protocol,
       byte[] namespace) throws ServiceException {
@@ -765,7 +775,9 @@ public class AccessControlUtil {
    * @param namespace name of the namespace
    * @param userName User name, if empty then all user permissions will be retrieved.
    * @throws ServiceException
+   * @deprecated Use {@link Admin#getUserPermissions(GetUserPermissionsRequest)} instead.
    */
+  @Deprecated
   public static List<UserPermission> getUserPermissions(RpcController controller,
       AccessControlService.BlockingInterface protocol, byte[] namespace, String userName)
       throws ServiceException {
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/GetUserPermissionsRequest.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/GetUserPermissionsRequest.java
new file mode 100644
index 0000000..8e1767c
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/GetUserPermissionsRequest.java
@@ -0,0 +1,136 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.security.access;
+
+import java.util.Objects;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Used by
+ * {@link org.apache.hadoop.hbase.client.Admin#getUserPermissions(GetUserPermissionsRequest)}.
+ * Represents the params of user permissions needed to get from HBase.
+ */
+@InterfaceAudience.Public
+public final class GetUserPermissionsRequest {
+  private String userName;
+  private String namespace;
+  private TableName tableName;
+  private byte[] family;
+  private byte[] qualifier;
+
+  private GetUserPermissionsRequest(String userName, String namespace, TableName tableName,
+      byte[] family, byte[] qualifier) {
+    this.userName = userName;
+    this.namespace = namespace;
+    this.tableName = tableName;
+    this.family = family;
+    this.qualifier = qualifier;
+  }
+
+  /**
+   * Build a get global permission request
+   * @return a get global permission request builder
+   */
+  public static Builder newBuilder() {
+    return new Builder();
+  }
+
+  /**
+   * Build a get namespace permission request
+   * @param namespace the specific namespace
+   * @return a get namespace permission request builder
+   */
+  public static Builder newBuilder(String namespace) {
+    return new Builder(namespace);
+  }
+
+  /**
+   * Build a get table permission request
+   * @param tableName the specific table name
+   * @return a get table permission request builder
+   */
+  public static Builder newBuilder(TableName tableName) {
+    return new Builder(tableName);
+  }
+
+  public String getUserName() {
+    return userName;
+  }
+
+  public String getNamespace() {
+    return namespace;
+  }
+
+  public TableName getTableName() {
+    return tableName;
+  }
+
+  public byte[] getFamily() {
+    return family;
+  }
+
+  public byte[] getQualifier() {
+    return qualifier;
+  }
+
+  public static final class Builder {
+    private String userName;
+    private String namespace;
+    private TableName tableName;
+    private byte[] family;
+    private byte[] qualifier;
+
+    private Builder() {
+    }
+
+    private Builder(String namespace) {
+      this.namespace = namespace;
+    }
+
+    private Builder(TableName tableName) {
+      this.tableName = tableName;
+    }
+
+    /**
+     * user name could be null if need all global/namespace/table permissions
+     */
+    public Builder withUserName(String userName) {
+      this.userName = userName;
+      return this;
+    }
+
+    public Builder withFamily(byte[] family) {
+      Objects.requireNonNull(tableName, "The tableName can't be NULL");
+      this.family = family;
+      return this;
+    }
+
+    public Builder withQualifier(byte[] qualifier) {
+      Objects.requireNonNull(tableName, "The tableName can't be NULL");
+      // Objects.requireNonNull(family, "The family can't be NULL");
+      this.qualifier = qualifier;
+      return this;
+    }
+
+    public GetUserPermissionsRequest build() {
+      return new GetUserPermissionsRequest(userName, namespace, tableName, family, qualifier);
+    }
+  }
+}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/ShadedAccessControlUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/ShadedAccessControlUtil.java
index b354d87..be3b75e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/ShadedAccessControlUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/ShadedAccessControlUtil.java
@@ -29,7 +29,9 @@ import org.apache.hbase.thirdparty.com.google.common.collect.ArrayListMultimap;
 import org.apache.hbase.thirdparty.com.google.common.collect.ListMultimap;
 import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.GetUserPermissionsResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.GrantRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.Permission.Type;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.RevokeRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
 
@@ -280,4 +282,40 @@ public class ShadedAccessControlUtil {
   public static RevokeRequest buildRevokeRequest(UserPermission userPermission) {
     return RevokeRequest.newBuilder().setUserPermission(toUserPermission(userPermission)).build();
   }
+
+  public static AccessControlProtos.GetUserPermissionsRequest
+      buildGetUserPermissionsRequest(GetUserPermissionsRequest request) {
+    AccessControlProtos.GetUserPermissionsRequest.Builder builder =
+        AccessControlProtos.GetUserPermissionsRequest.newBuilder();
+    if (request.getUserName() != null && !request.getUserName().isEmpty()) {
+      builder.setUserName(ByteString.copyFromUtf8(request.getUserName()));
+    }
+    if (request.getNamespace() != null && !request.getNamespace().isEmpty()) {
+      builder.setNamespaceName(ByteString.copyFromUtf8(request.getNamespace()));
+      builder.setType(Type.Namespace);
+    }
+    if (request.getTableName() != null) {
+      builder.setTableName(toProtoTableName(request.getTableName()));
+      builder.setType(Type.Table);
+    }
+    if (!builder.hasType()) {
+      builder.setType(Type.Global);
+    }
+    if (request.getFamily() != null && request.getFamily().length > 0) {
+      builder.setColumnFamily(ByteString.copyFrom(request.getFamily()));
+    }
+    if (request.getQualifier() != null && request.getQualifier().length > 0) {
+      builder.setColumnQualifier(ByteString.copyFrom(request.getQualifier()));
+    }
+    return builder.build();
+  }
+
+  public static GetUserPermissionsResponse
+      buildGetUserPermissionsResponse(final List<UserPermission> permissions) {
+    GetUserPermissionsResponse.Builder builder = GetUserPermissionsResponse.newBuilder();
+    for (UserPermission perm : permissions) {
+      builder.addUserPermission(toUserPermission(perm));
+    }
+    return builder.build();
+  }
 }
diff --git a/hbase-protocol-shaded/src/main/protobuf/AccessControl.proto b/hbase-protocol-shaded/src/main/protobuf/AccessControl.proto
index 3b0e9bb..af60fe2 100644
--- a/hbase-protocol-shaded/src/main/protobuf/AccessControl.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/AccessControl.proto
@@ -103,6 +103,9 @@ message GetUserPermissionsRequest {
   optional Permission.Type type = 1;
   optional TableName table_name = 2;
   optional bytes namespace_name = 3;
+  optional bytes column_family = 4;
+  optional bytes column_qualifier = 5;
+  optional bytes user_name = 6;
 }
 
 message GetUserPermissionsResponse {
diff --git a/hbase-protocol-shaded/src/main/protobuf/Master.proto b/hbase-protocol-shaded/src/main/protobuf/Master.proto
index 4ed0ad5..d883b4c 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Master.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Master.proto
@@ -1031,6 +1031,8 @@ service MasterService {
   rpc Grant(GrantRequest) returns (GrantResponse);
 
   rpc Revoke(RevokeRequest) returns (RevokeResponse);
+
+  rpc GetUserPermissions (GetUserPermissionsRequest) returns (GetUserPermissionsResponse);
 }
 
 // HBCK Service definitions.
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
index d8b7d4d..bfb1ada 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
@@ -1632,4 +1632,34 @@ public interface MasterObserver {
   default void postRevoke(ObserverContext<MasterCoprocessorEnvironment> ctx,
       UserPermission userPermission) throws IOException {
   }
+
+  /**
+   * Called before getting user permissions.
+   * @param ctx the coprocessor instance's environment
+   * @param userName the user name, null if get all user permissions
+   * @param namespace the namespace, null if don't get namespace permission
+   * @param tableName the table name, null if don't get table permission
+   * @param family the table column family, null if don't get table family permission
+   * @param qualifier the table column qualifier, null if don't get table qualifier permission
+   * @throws IOException if something went wrong
+   */
+  default void preGetUserPermissions(ObserverContext<MasterCoprocessorEnvironment> ctx,
+      String userName, String namespace, TableName tableName, byte[] family, byte[] qualifier)
+      throws IOException {
+  }
+
+  /**
+   * Called after getting user permissions.
+   * @param ctx the coprocessor instance's environment
+   * @param userName the user name, null if get all user permissions
+   * @param namespace the namespace, null if don't get namespace permission
+   * @param tableName the table name, null if don't get table permission
+   * @param family the table column family, null if don't get table family permission
+   * @param qualifier the table column qualifier, null if don't get table qualifier permission
+   * @throws IOException if something went wrong
+   */
+  default void postGetUserPermissions(ObserverContext<MasterCoprocessorEnvironment> ctx,
+      String userName, String namespace, TableName tableName, byte[] family, byte[] qualifier)
+      throws IOException {
+  }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
index 8764143..bb9fc3f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
@@ -1892,4 +1892,24 @@ public class MasterCoprocessorHost
       }
     });
   }
+
+  public void preGetUserPermissions(String userName, String namespace, TableName tableName,
+      byte[] family, byte[] qualifier) throws IOException {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
+      @Override
+      public void call(MasterObserver observer) throws IOException {
+        observer.preGetUserPermissions(this, userName, namespace, tableName, family, qualifier);
+      }
+    });
+  }
+
+  public void postGetUserPermissions(String userName, String namespace, TableName tableName,
+      byte[] family, byte[] qualifier) throws IOException {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
+      @Override
+      public void call(MasterObserver observer) throws IOException {
+        observer.postGetUserPermissions(this, userName, namespace, tableName, family, qualifier);
+      }
+    });
+  }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index b943000..08f89db 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -91,11 +91,13 @@ import org.apache.hadoop.hbase.regionserver.RpcSchedulerFactory;
 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.security.Superusers;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.access.AccessChecker;
 import org.apache.hadoop.hbase.security.access.AccessControlLists;
 import org.apache.hadoop.hbase.security.access.AccessController;
 import org.apache.hadoop.hbase.security.access.Permission;
+import org.apache.hadoop.hbase.security.access.Permission.Action;
 import org.apache.hadoop.hbase.security.access.ShadedAccessControlUtil;
 import org.apache.hadoop.hbase.security.access.UserPermission;
 import org.apache.hadoop.hbase.security.visibility.VisibilityController;
@@ -118,8 +120,12 @@ import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
 
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.GetUserPermissionsRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.GetUserPermissionsResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.GrantRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.GrantResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.Permission.Type;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.RevokeRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.RevokeResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest;
@@ -2571,6 +2577,57 @@ public class MasterRpcServices extends RSRpcServices
     }
   }
 
+  @Override
+  public GetUserPermissionsResponse getUserPermissions(RpcController controller,
+      GetUserPermissionsRequest request) throws ServiceException {
+    try {
+      final String userName = request.hasUserName() ? request.getUserName().toStringUtf8() : null;
+      String namespace =
+          request.hasNamespaceName() ? request.getNamespaceName().toStringUtf8() : null;
+      TableName table =
+          request.hasTableName() ? ProtobufUtil.toTableName(request.getTableName()) : null;
+      byte[] cf = request.hasColumnFamily() ? request.getColumnFamily().toByteArray() : null;
+      byte[] cq = request.hasColumnQualifier() ? request.getColumnQualifier().toByteArray() : null;
+      Type permissionType = request.hasType() ? request.getType() : null;
+      if (master.cpHost != null) {
+        master.getMasterCoprocessorHost().preGetUserPermissions(userName, namespace, table, cf, cq);
+      }
+
+      List<UserPermission> perms = null;
+      if (permissionType == Type.Table) {
+        boolean filter = (cf != null || userName != null) ? true : false;
+        perms = AccessControlLists.getUserTablePermissions(master.getConfiguration(), table, cf, cq,
+          userName, filter);
+      } else if (permissionType == Type.Namespace) {
+        perms = AccessControlLists.getUserNamespacePermissions(master.getConfiguration(), namespace,
+          userName, userName != null ? true : false);
+      } else {
+        perms = AccessControlLists.getUserPermissions(master.getConfiguration(), null, null, null,
+          userName, userName != null ? true : false);
+        // Skip super users when filter user is specified
+        if (userName == null) {
+          // Adding superusers explicitly to the result set as AccessControlLists do not store
+          // them. Also using acl as table name to be inline with the results of global admin and
+          // will help in avoiding any leakage of information about being superusers.
+          for (String user : Superusers.getSuperUsers()) {
+            perms.add(new UserPermission(user,
+                Permission.newBuilder().withActions(Action.values()).build()));
+          }
+        }
+      }
+
+      if (master.cpHost != null) {
+        master.getMasterCoprocessorHost().postGetUserPermissions(userName, namespace, table, cf,
+          cq);
+      }
+      AccessControlProtos.GetUserPermissionsResponse response =
+          ShadedAccessControlUtil.buildGetUserPermissionsResponse(perms);
+      return response;
+    } catch (IOException ioe) {
+      throw new ServiceException(ioe);
+    }
+  }
+
   private boolean containMetaWals(ServerName serverName) throws IOException {
     Path logDir = new Path(master.getWALRootDir(),
         AbstractFSWALProvider.getWALDirectoryName(serverName.toString()));
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java
index 74c848d..1182baa 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java
@@ -538,8 +538,9 @@ public class AccessControlLists {
    * Returns the currently granted permissions for a given table as the specified user plus
    * associated permissions.
    */
-  static List<UserPermission> getUserTablePermissions(Configuration conf, TableName tableName,
-      byte[] cf, byte[] cq, String userName, boolean hasFilterUser) throws IOException {
+  public static List<UserPermission> getUserTablePermissions(Configuration conf,
+      TableName tableName, byte[] cf, byte[] cq, String userName, boolean hasFilterUser)
+      throws IOException {
     return getUserPermissions(conf, tableName == null ? null : tableName.getName(), cf, cq,
       userName, hasFilterUser);
   }
@@ -548,8 +549,8 @@ public class AccessControlLists {
    * Returns the currently granted permissions for a given namespace as the specified user plus
    * associated permissions.
    */
-  static List<UserPermission> getUserNamespacePermissions(Configuration conf, String namespace,
-      String user, boolean hasFilterUser) throws IOException {
+  public static List<UserPermission> getUserNamespacePermissions(Configuration conf,
+      String namespace, String user, boolean hasFilterUser) throws IOException {
     return getUserPermissions(conf, Bytes.toBytes(toNamespaceEntry(namespace)), null, null, user,
       hasFilterUser);
   }
@@ -566,8 +567,8 @@ public class AccessControlLists {
    * @return List of UserPermissions
    * @throws IOException on failure
    */
-  static List<UserPermission> getUserPermissions(Configuration conf, byte[] entryName, byte[] cf,
-      byte[] cq, String user, boolean hasFilterUser) throws IOException {
+  public static List<UserPermission> getUserPermissions(Configuration conf, byte[] entryName,
+      byte[] cf, byte[] cq, String user, boolean hasFilterUser) throws IOException {
     ListMultimap<String, UserPermission> allPerms =
         getPermissions(conf, entryName, null, cf, cq, user, hasFilterUser);
     List<UserPermission> perms = new ArrayList<>();
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
index a4a96c0..aaa2da5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
@@ -2135,6 +2135,10 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor,
     done.run(response);
   }
 
+  /**
+   * @deprecated Use {@link Admin#getUserPermissions(GetUserPermissionsRequest)} instead.
+   */
+  @Deprecated
   @Override
   public void getUserPermissions(RpcController controller,
       AccessControlProtos.GetUserPermissionsRequest request,
@@ -2147,77 +2151,29 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor,
           throw new CoprocessorException("AccessController not yet initialized");
         }
         User caller = RpcServer.getRequestUser().orElse(null);
-
-        List<UserPermission> perms = null;
-        // Initialize username, cf and cq. Set to null if request doesn't have.
         final String userName = request.hasUserName() ? request.getUserName().toStringUtf8() : null;
+        final String namespace =
+            request.hasNamespaceName() ? request.getNamespaceName().toStringUtf8() : null;
+        final TableName table =
+            request.hasTableName() ? ProtobufUtil.toTableName(request.getTableName()) : null;
         final byte[] cf =
             request.hasColumnFamily() ? request.getColumnFamily().toByteArray() : null;
         final byte[] cq =
             request.hasColumnQualifier() ? request.getColumnQualifier().toByteArray() : null;
-
+        preGetUserPermissions(caller, userName, namespace, table, cf, cq);
+        GetUserPermissionsRequest getUserPermissionsRequest = null;
         if (request.getType() == AccessControlProtos.Permission.Type.Table) {
-          final TableName table = request.hasTableName() ?
-            ProtobufUtil.toTableName(request.getTableName()) : null;
-          accessChecker.requirePermission(caller, "userPermissions", table, cf, cq, userName,
-            Action.ADMIN);
-          perms = User.runAsLoginUser(new PrivilegedExceptionAction<List<UserPermission>>() {
-            @Override
-            public List<UserPermission> run() throws Exception {
-              if (cf != null || userName != null) {
-                // retrieve permission based on the requested parameters
-                return AccessControlLists.getUserTablePermissions(regionEnv.getConfiguration(),
-                  table, cf, cq, userName, true);
-              } else {
-                return AccessControlLists.getUserTablePermissions(regionEnv.getConfiguration(),
-                  table, null, null, null, false);
-              }
-            }
-          });
+          getUserPermissionsRequest = GetUserPermissionsRequest.newBuilder(table).withFamily(cf)
+              .withQualifier(cq).withUserName(userName).build();
         } else if (request.getType() == AccessControlProtos.Permission.Type.Namespace) {
-          final String namespace = request.getNamespaceName().toStringUtf8();
-          accessChecker.requireNamespacePermission(caller, "userPermissions",
-            namespace, userName, Action.ADMIN);
-          perms = User.runAsLoginUser(new PrivilegedExceptionAction<List<UserPermission>>() {
-            @Override
-            public List<UserPermission> run() throws Exception {
-              if (userName != null) {
-                // retrieve permission based on the requested parameters
-                return AccessControlLists.getUserNamespacePermissions(regionEnv.getConfiguration(),
-                  namespace, userName, true);
-              } else {
-                return AccessControlLists.getUserNamespacePermissions(regionEnv.getConfiguration(),
-                  namespace, null, false);
-              }
-            }
-          });
+          getUserPermissionsRequest =
+              GetUserPermissionsRequest.newBuilder(namespace).withUserName(userName).build();
         } else {
-          accessChecker.requirePermission(caller, "userPermissions", userName, Action.ADMIN);
-          perms = User.runAsLoginUser(new PrivilegedExceptionAction<List<UserPermission>>() {
-            @Override
-            public List<UserPermission> run() throws Exception {
-              if (userName != null) {
-                // retrieve permission based on the requested parameters
-                return AccessControlLists.getUserPermissions(regionEnv.getConfiguration(), null,
-                  null, null, userName, true);
-              } else {
-                return AccessControlLists.getUserPermissions(regionEnv.getConfiguration(), null,
-                  null, null, null, false);
-              }
-            }
-          });
-
-          // Skip super users when filter user is specified
-          if (userName == null) {
-            // Adding superusers explicitly to the result set as AccessControlLists do not store
-            // them. Also using acl as table name to be inline with the results of global admin and
-            // will help in avoiding any leakage of information about being superusers.
-            for (String user : Superusers.getSuperUsers()) {
-              perms.add(new UserPermission(user,
-                  Permission.newBuilder().withActions(Action.values()).build()));
-            }
-          }
+          getUserPermissionsRequest =
+              GetUserPermissionsRequest.newBuilder().withUserName(userName).build();
         }
+        List<UserPermission> perms =
+            regionEnv.getConnection().getAdmin().getUserPermissions(getUserPermissionsRequest);
         response = AccessControlUtil.buildGetUserPermissionsResponse(perms);
       } else {
         throw new CoprocessorException(AccessController.class, "This method "
@@ -2680,4 +2636,24 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor,
       accessChecker.performOnSuperuser(request, caller, userPermission.getUser());
     }
   }
+
+  @Override
+  public void preGetUserPermissions(ObserverContext<MasterCoprocessorEnvironment> ctx,
+      String userName, String namespace, TableName tableName, byte[] family, byte[] qualifier)
+      throws IOException {
+    preGetUserPermissions(getActiveUser(ctx), userName, namespace, tableName, family, qualifier);
+  }
+
+  private void preGetUserPermissions(User caller, String userName, String namespace,
+      TableName tableName, byte[] family, byte[] qualifier) throws IOException {
+    if (tableName != null) {
+      accessChecker.requirePermission(caller, "getUserPermissions", tableName, family, qualifier,
+        userName, Action.ADMIN);
+    } else if (namespace != null) {
+      accessChecker.requireNamespacePermission(caller, "getUserPermissions", namespace, userName,
+        Action.ADMIN);
+    } else {
+      accessChecker.requirePermission(caller, "getUserPermissions", userName, Action.ADMIN);
+    }
+  }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAccessControlAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAccessControlAdminApi.java
new file mode 100644
index 0000000..8075ae0
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAccessControlAdminApi.java
@@ -0,0 +1,74 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at http://www.apache.org/licenses/LICENSE-2.0 Unless required by applicable
+ * law or agreed to in writing, software distributed under the License is distributed on an "AS IS"
+ * BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License
+ * for the specific language governing permissions and limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.client;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.List;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.security.access.AccessControlLists;
+import org.apache.hadoop.hbase.security.access.GetUserPermissionsRequest;
+import org.apache.hadoop.hbase.security.access.Permission;
+import org.apache.hadoop.hbase.security.access.SecureTestUtil;
+import org.apache.hadoop.hbase.security.access.UserPermission;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+@Category({ ClientTests.class, SmallTests.class })
+public class TestAsyncAccessControlAdminApi extends TestAsyncAdminBase {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestAsyncAccessControlAdminApi.class);
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    SecureTestUtil.enableSecurity(TEST_UTIL.getConfiguration());
+    TEST_UTIL.startMiniCluster(1);
+    TEST_UTIL.waitTableAvailable(AccessControlLists.ACL_TABLE_NAME);
+    ASYNC_CONN = ConnectionFactory.createAsyncConnection(TEST_UTIL.getConfiguration()).get();
+  }
+
+  @Test
+  public void testGrant() throws Exception {
+    TableName tableName = TableName.valueOf("test-table");
+    String user = "test-user";
+    UserPermission userPermission = new UserPermission(user,
+        Permission.newBuilder(tableName).withActions(Permission.Action.READ).build());
+    // grant user table permission
+    admin.grant(userPermission, false).get();
+
+    // get table permissions
+    List<UserPermission> userPermissions =
+        admin.getUserPermissions(GetUserPermissionsRequest.newBuilder(tableName).build()).get();
+    assertEquals(1, userPermissions.size());
+    assertEquals(userPermission, userPermissions.get(0));
+
+    // get user table permissions
+    userPermissions = admin.getUserPermissions(
+      GetUserPermissionsRequest.newBuilder(tableName).withUserName(user).build()).get();
+    assertEquals(1, userPermissions.size());
+    assertEquals(userPermission, userPermissions.get(0));
+
+    userPermissions = admin.getUserPermissions(
+      GetUserPermissionsRequest.newBuilder(tableName).withUserName("u").build()).get();
+    assertEquals(0, userPermissions.size());
+  }
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java
index 129f2b6..eecd773 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java
@@ -144,7 +144,8 @@ public class SecureTestUtil {
       Permission.Action... actions) throws IOException {
     Permission[] perms = new Permission[actions.length];
     for (int i = 0; i < actions.length; i++) {
-      perms[i] = new TablePermission(table, family, column, actions[i]);
+      perms[i] = Permission.newBuilder(table).withFamily(family).withQualifier(column)
+          .withActions(actions[i]).build();
     }
 
     checkTablePerms(conf, table, perms);
@@ -878,7 +879,8 @@ public class SecureTestUtil {
       byte[] column, Permission.Action... actions) throws IOException {
     Permission[] perms = new Permission[actions.length];
     for (int i = 0; i < actions.length; i++) {
-      perms[i] = new TablePermission(table, family, column, actions[i]);
+      perms[i] = Permission.newBuilder(table).withFamily(family).withQualifier(column)
+          .withActions(actions[i]).build();
     }
     checkTablePerms(testUtil, table, perms);
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
index 0d7a0e0..563238c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
@@ -1193,12 +1193,9 @@ public class TestAccessController extends SecureTestUtil {
     AccessTestAction getTablePermissionsAction = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
-        try(Connection conn = ConnectionFactory.createConnection(conf);
-            Table acl = conn.getTable(AccessControlLists.ACL_TABLE_NAME)){
-          BlockingRpcChannel service = acl.coprocessorService(TEST_TABLE.getName());
-          AccessControlService.BlockingInterface protocol =
-              AccessControlService.newBlockingStub(service);
-          AccessControlUtil.getUserPermissions(null, protocol, TEST_TABLE);
+        try (Connection conn = ConnectionFactory.createConnection(conf)) {
+          conn.getAdmin()
+              .getUserPermissions(GetUserPermissionsRequest.newBuilder(TEST_TABLE).build());
         }
         return null;
       }
@@ -1207,12 +1204,9 @@ public class TestAccessController extends SecureTestUtil {
     AccessTestAction getGlobalPermissionsAction = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
-        try(Connection conn = ConnectionFactory.createConnection(conf);
+        try (Connection conn = ConnectionFactory.createConnection(conf);
             Table acl = conn.getTable(AccessControlLists.ACL_TABLE_NAME)) {
-          BlockingRpcChannel service = acl.coprocessorService(HConstants.EMPTY_START_ROW);
-          AccessControlService.BlockingInterface protocol =
-            AccessControlService.newBlockingStub(service);
-          AccessControlUtil.getUserPermissions(null, protocol);
+          conn.getAdmin().getUserPermissions(GetUserPermissionsRequest.newBuilder().build());
         }
         return null;
       }
@@ -1682,17 +1676,8 @@ public class TestAccessController extends SecureTestUtil {
     htd.setOwner(USER_OWNER);
     createTable(TEST_UTIL, htd);
     try {
-      List<UserPermission> perms;
-      Table acl = systemUserConnection.getTable(AccessControlLists.ACL_TABLE_NAME);
-      try {
-        BlockingRpcChannel service = acl.coprocessorService(tableName.getName());
-        AccessControlService.BlockingInterface protocol =
-            AccessControlService.newBlockingStub(service);
-        perms = AccessControlUtil.getUserPermissions(null, protocol, tableName);
-      } finally {
-        acl.close();
-      }
-
+      List<UserPermission> perms =
+          admin.getUserPermissions(GetUserPermissionsRequest.newBuilder(tableName).build());
       UserPermission ownerperm = new UserPermission(USER_OWNER.getName(),
           Permission.newBuilder(tableName).withActions(Action.values()).build());
       assertTrue("Owner should have all permissions on table",
@@ -1711,16 +1696,7 @@ public class TestAccessController extends SecureTestUtil {
       grantOnTable(TEST_UTIL, user.getShortName(), tableName, family1, qualifier,
         Permission.Action.READ);
 
-      acl = systemUserConnection.getTable(AccessControlLists.ACL_TABLE_NAME);
-      try {
-        BlockingRpcChannel service = acl.coprocessorService(tableName.getName());
-        AccessControlService.BlockingInterface protocol =
-            AccessControlService.newBlockingStub(service);
-        perms = AccessControlUtil.getUserPermissions(null, protocol, tableName);
-      } finally {
-        acl.close();
-      }
-
+      perms = admin.getUserPermissions(GetUserPermissionsRequest.newBuilder(tableName).build());
       UserPermission upToVerify =
           new UserPermission(userName, Permission.newBuilder(tableName).withFamily(family1)
               .withQualifier(qualifier).withActions(Permission.Action.READ).build());
@@ -1736,16 +1712,7 @@ public class TestAccessController extends SecureTestUtil {
       grantOnTable(TEST_UTIL, user.getShortName(), tableName, family1, qualifier,
         Permission.Action.WRITE, Permission.Action.READ);
 
-      acl = systemUserConnection.getTable(AccessControlLists.ACL_TABLE_NAME);
-      try {
-        BlockingRpcChannel service = acl.coprocessorService(tableName.getName());
-        AccessControlService.BlockingInterface protocol =
-            AccessControlService.newBlockingStub(service);
-        perms = AccessControlUtil.getUserPermissions(null, protocol, tableName);
-      } finally {
-        acl.close();
-      }
-
+      perms = admin.getUserPermissions(GetUserPermissionsRequest.newBuilder(tableName).build());
       upToVerify = new UserPermission(userName,
           Permission.newBuilder(tableName).withFamily(family1).withQualifier(qualifier)
               .withActions(Permission.Action.WRITE, Permission.Action.READ).build());
@@ -1756,16 +1723,7 @@ public class TestAccessController extends SecureTestUtil {
       revokeFromTable(TEST_UTIL, user.getShortName(), tableName, family1, qualifier,
         Permission.Action.WRITE, Permission.Action.READ);
 
-      acl = systemUserConnection.getTable(AccessControlLists.ACL_TABLE_NAME);
-      try {
-        BlockingRpcChannel service = acl.coprocessorService(tableName.getName());
-        AccessControlService.BlockingInterface protocol =
-            AccessControlService.newBlockingStub(service);
-        perms = AccessControlUtil.getUserPermissions(null, protocol, tableName);
-      } finally {
-        acl.close();
-      }
-
+      perms = admin.getUserPermissions(GetUserPermissionsRequest.newBuilder(tableName).build());
       assertFalse("User should not be granted permission: " + upToVerify.toString(),
         hasFoundUserPermission(upToVerify, perms));
 
@@ -1776,16 +1734,7 @@ public class TestAccessController extends SecureTestUtil {
       htd.setOwner(newOwner);
       admin.modifyTable(htd);
 
-      acl = systemUserConnection.getTable(AccessControlLists.ACL_TABLE_NAME);
-      try {
-        BlockingRpcChannel service = acl.coprocessorService(tableName.getName());
-        AccessControlService.BlockingInterface protocol =
-            AccessControlService.newBlockingStub(service);
-        perms = AccessControlUtil.getUserPermissions(null, protocol, tableName);
-      } finally {
-        acl.close();
-      }
-
+      perms = admin.getUserPermissions(GetUserPermissionsRequest.newBuilder(tableName).build());
       UserPermission newOwnerperm = new UserPermission(newOwner.getName(),
           Permission.newBuilder(tableName).withActions(Action.values()).build());
       assertTrue("New owner should have all permissions on table",
@@ -1798,16 +1747,8 @@ public class TestAccessController extends SecureTestUtil {
 
   @Test
   public void testGlobalPermissionList() throws Exception {
-    List<UserPermission> perms;
-    Table acl = systemUserConnection.getTable(AccessControlLists.ACL_TABLE_NAME);
-    try {
-      BlockingRpcChannel service = acl.coprocessorService(HConstants.EMPTY_START_ROW);
-      AccessControlService.BlockingInterface protocol =
-        AccessControlService.newBlockingStub(service);
-      perms = AccessControlUtil.getUserPermissions(null, protocol);
-    } finally {
-      acl.close();
-    }
+    List<UserPermission> perms = systemUserConnection.getAdmin()
+        .getUserPermissions(GetUserPermissionsRequest.newBuilder().build());
 
     Collection<String> superUsers = Superusers.getSuperUsers();
     List<UserPermission> adminPerms = new ArrayList<>(superUsers.size() + 1);
@@ -1903,9 +1844,12 @@ public class TestAccessController extends SecureTestUtil {
       AccessTestAction multiQualifierRead = new AccessTestAction() {
         @Override
         public Void run() throws Exception {
-          checkTablePerms(TEST_UTIL, TEST_TABLE, new Permission[] {
-              new TablePermission(TEST_TABLE, TEST_FAMILY, TEST_Q1, Permission.Action.READ),
-              new TablePermission(TEST_TABLE, TEST_FAMILY, TEST_Q2, Permission.Action.READ), });
+          checkTablePerms(TEST_UTIL, TEST_TABLE,
+            new Permission[] {
+                Permission.newBuilder(TEST_TABLE).withFamily(TEST_FAMILY).withQualifier(TEST_Q1)
+                    .withActions(Permission.Action.READ).build(),
+                Permission.newBuilder(TEST_TABLE).withFamily(TEST_FAMILY).withQualifier(TEST_Q2)
+                    .withActions(Permission.Action.READ).build(), });
           return null;
         }
       };
@@ -1915,7 +1859,7 @@ public class TestAccessController extends SecureTestUtil {
         public Void run() throws Exception {
           checkTablePerms(TEST_UTIL, TEST_TABLE, new Permission[] {
               new Permission(Permission.Action.READ),
-              new TablePermission(TEST_TABLE, null, (byte[]) null, Permission.Action.READ), });
+              Permission.newBuilder(TEST_TABLE).withActions(Permission.Action.READ).build() });
           return null;
         }
       };
@@ -3545,12 +3489,9 @@ public class TestAccessController extends SecureTestUtil {
     AccessTestAction globalUserPermissionAction = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
-        try (Connection conn = ConnectionFactory.createConnection(conf);
-            Table acl = conn.getTable(AccessControlLists.ACL_TABLE_NAME)) {
-          BlockingRpcChannel service = acl.coprocessorService(TEST_TABLE.getName());
-          AccessControlService.BlockingInterface protocol =
-              AccessControlService.newBlockingStub(service);
-          AccessControlUtil.getUserPermissions(null, protocol, "dummy");
+        try (Connection conn = ConnectionFactory.createConnection(conf)) {
+          conn.getAdmin().getUserPermissions(
+            GetUserPermissionsRequest.newBuilder().withUserName("dummy").build());
         }
         return null;
       }
@@ -3584,12 +3525,9 @@ public class TestAccessController extends SecureTestUtil {
     AccessTestAction namespaceUserPermissionAction = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
-        try (Connection conn = ConnectionFactory.createConnection(conf);
-            Table acl = conn.getTable(AccessControlLists.ACL_TABLE_NAME)) {
-          BlockingRpcChannel service = acl.coprocessorService(TEST_TABLE.getName());
-          AccessControlService.BlockingInterface protocol =
-              AccessControlService.newBlockingStub(service);
-          AccessControlUtil.getUserPermissions(null, protocol, Bytes.toBytes(namespace1), "dummy");
+        try (Connection conn = ConnectionFactory.createConnection(conf)) {
+          conn.getAdmin().getUserPermissions(
+            GetUserPermissionsRequest.newBuilder(namespace1).withUserName("dummy").build());
         }
         return null;
       }
@@ -3628,13 +3566,9 @@ public class TestAccessController extends SecureTestUtil {
     AccessTestAction tableUserPermissionAction = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
-        try (Connection conn = ConnectionFactory.createConnection(conf);
-            Table acl = conn.getTable(AccessControlLists.ACL_TABLE_NAME)) {
-          BlockingRpcChannel service = acl.coprocessorService(TEST_TABLE.getName());
-          AccessControlService.BlockingInterface protocol =
-              AccessControlService.newBlockingStub(service);
-          AccessControlUtil.getUserPermissions(null, protocol, TEST_TABLE, TEST_FAMILY,
-            TEST_QUALIFIER, "dummy");
+        try (Connection conn = ConnectionFactory.createConnection(conf)) {
+          conn.getAdmin().getUserPermissions(GetUserPermissionsRequest.newBuilder(TEST_TABLE)
+              .withFamily(TEST_FAMILY).withQualifier(TEST_QUALIFIER).withUserName("dummy").build());
         }
         return null;
       }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java
index 82d0f6e..f6b018e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java
@@ -406,12 +406,9 @@ public class TestNamespaceCommands extends SecureTestUtil {
     AccessTestAction getPermissionsAction = new AccessTestAction() {
       @Override
       public Object run() throws Exception {
-        try (Connection connection = ConnectionFactory.createConnection(conf);
-            Table acl = connection.getTable(AccessControlLists.ACL_TABLE_NAME)) {
-          BlockingRpcChannel service = acl.coprocessorService(HConstants.EMPTY_START_ROW);
-          AccessControlService.BlockingInterface protocol =
-              AccessControlService.newBlockingStub(service);
-          AccessControlUtil.getUserPermissions(null, protocol, Bytes.toBytes(TEST_NAMESPACE));
+        try (Connection connection = ConnectionFactory.createConnection(conf)) {
+          connection.getAdmin()
+              .getUserPermissions(GetUserPermissionsRequest.newBuilder(TEST_NAMESPACE).build());
         }
         return null;
       }
@@ -421,7 +418,8 @@ public class TestNamespaceCommands extends SecureTestUtil {
       @Override
       public Object run() throws Exception {
         ACCESS_CONTROLLER.preGrant(ObserverContextImpl.createAndPrepare(CP_ENV),
-          new UserPermission(testUser, new NamespacePermission(TEST_NAMESPACE, Action.WRITE)),
+          new UserPermission(testUser,
+              Permission.newBuilder(TEST_NAMESPACE).withActions(Action.WRITE).build()),
           false);
         return null;
       }
@@ -430,7 +428,8 @@ public class TestNamespaceCommands extends SecureTestUtil {
       @Override
       public Object run() throws Exception {
         ACCESS_CONTROLLER.preRevoke(ObserverContextImpl.createAndPrepare(CP_ENV),
-          new UserPermission(testUser, new NamespacePermission(TEST_NAMESPACE, Action.WRITE)));
+          new UserPermission(testUser,
+              Permission.newBuilder(TEST_NAMESPACE).withActions(Action.WRITE).build()));
         return null;
       }
     };
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java
index 85adf7d..7b96a28 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java
@@ -56,6 +56,7 @@ import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.replication.SyncReplicationState;
+import org.apache.hadoop.hbase.security.access.GetUserPermissionsRequest;
 import org.apache.hadoop.hbase.security.access.UserPermission;
 import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
 import org.apache.hadoop.hbase.thrift2.ThriftUtilities;
@@ -1104,4 +1105,10 @@ public class ThriftAdmin implements Admin {
   public void revoke(UserPermission userPermission) {
     throw new NotImplementedException("revoke not supported in ThriftAdmin");
   }
+
+  @Override
+  public List<UserPermission>
+      getUserPermissions(GetUserPermissionsRequest getUserPermissionsRequest) {
+    throw new NotImplementedException("getUserPermissions not supported in ThriftAdmin");
+  }
 }


[hbase] 32/49: HBASE-21964 unset Quota by Throttle Type

Posted by op...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

openinx pushed a commit to branch HBASE-21879
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit e694a4a525145be302be19871b768c597b606763
Author: yaojingyi <ya...@didichuxing.com>
AuthorDate: Fri Mar 22 10:55:03 2019 +0800

    HBASE-21964 unset Quota by Throttle Type
    
    Signed-off-by: Guanghao Zhang <zg...@apache.org>
---
 .../hadoop/hbase/quotas/QuotaSettingsFactory.java  |  74 ++++++++++
 .../hbase/quotas/GlobalQuotaSettingsImpl.java      | 122 +++++++++++++++--
 .../hadoop/hbase/quotas/TestQuotaThrottle.java     | 151 +++++++++++++++++++++
 hbase-shell/src/main/ruby/hbase/quotas.rb          | 146 ++++++++++++++++----
 4 files changed, 455 insertions(+), 38 deletions(-)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSettingsFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSettingsFactory.java
index 3124591..4473b73 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSettingsFactory.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaSettingsFactory.java
@@ -291,6 +291,18 @@ public class QuotaSettingsFactory {
   }
 
   /**
+   * Remove the throttling for the specified user.
+   *
+   * @param userName the user
+   * @param type the type of throttling
+   * @return the quota settings
+   */
+  public static QuotaSettings unthrottleUserByThrottleType(final String userName,
+      final ThrottleType type) {
+    return throttle(userName, null, null, null, type, 0, null, QuotaScope.MACHINE);
+  }
+
+  /**
    * Remove the throttling for the specified user on the specified table.
    *
    * @param userName the user
@@ -302,6 +314,19 @@ public class QuotaSettingsFactory {
   }
 
   /**
+   * Remove the throttling for the specified user on the specified table.
+   *
+   * @param userName the user
+   * @param tableName the table
+   * @param type the type of throttling
+   * @return the quota settings
+   */
+  public static QuotaSettings unthrottleUserByThrottleType(final String userName,
+      final TableName tableName, final ThrottleType type) {
+    return throttle(userName, tableName, null, null, type, 0, null, QuotaScope.MACHINE);
+  }
+
+  /**
    * Remove the throttling for the specified user on the specified namespace.
    *
    * @param userName the user
@@ -313,6 +338,19 @@ public class QuotaSettingsFactory {
   }
 
   /**
+   * Remove the throttling for the specified user on the specified namespace.
+   *
+   * @param userName the user
+   * @param namespace the namespace
+   * @param type the type of throttling
+   * @return the quota settings
+   */
+  public static QuotaSettings unthrottleUserByThrottleType(final String userName,
+      final String namespace, final ThrottleType type) {
+    return throttle(userName, null, namespace, null, type, 0, null, QuotaScope.MACHINE);
+  }
+
+  /**
    * Throttle the specified table.
    *
    * @param tableName the table to throttle
@@ -351,6 +389,18 @@ public class QuotaSettingsFactory {
   }
 
   /**
+   * Remove the throttling for the specified table.
+   *
+   * @param tableName the table
+   * @param type the type of throttling
+   * @return the quota settings
+   */
+  public static QuotaSettings unthrottleTableByThrottleType(final TableName tableName,
+      final ThrottleType type) {
+    return throttle(null, tableName, null, null, type, 0, null, QuotaScope.MACHINE);
+  }
+
+  /**
    * Throttle the specified namespace.
    *
    * @param namespace the namespace to throttle
@@ -389,6 +439,18 @@ public class QuotaSettingsFactory {
   }
 
   /**
+   * Remove the throttling for the specified namespace by throttle type.
+   *
+   * @param namespace the namespace
+   * @param type the type of throttling
+   * @return the quota settings
+   */
+  public static QuotaSettings unthrottleNamespaceByThrottleType(final String namespace,
+      final ThrottleType type) {
+    return throttle(null, null, namespace, null, type, 0, null, QuotaScope.MACHINE);
+  }
+
+  /**
    * Throttle the specified region server.
    *
    * @param regionServer the region server to throttle
@@ -412,6 +474,18 @@ public class QuotaSettingsFactory {
     return throttle(null, null, null, regionServer, null, 0, null, QuotaScope.MACHINE);
   }
 
+  /**
+   * Remove the throttling for the specified region server by throttle type.
+   *
+   * @param regionServer  the region Server
+   * @param type the type of throttling
+   * @return the quota settings
+   */
+  public static QuotaSettings unthrottleRegionServerByThrottleType(final String regionServer,
+      final ThrottleType type) {
+    return throttle(null, null, null, regionServer, type, 0, null, QuotaScope.MACHINE);
+  }
+
   /* Throttle helper */
   private static QuotaSettings throttle(final String userName, final TableName tableName,
       final String namespace, final String regionServer, final ThrottleType type, final long limit,
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/GlobalQuotaSettingsImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/GlobalQuotaSettingsImpl.java
index c7df789..6bcbf7e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/GlobalQuotaSettingsImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/GlobalQuotaSettingsImpl.java
@@ -110,6 +110,60 @@ public class GlobalQuotaSettingsImpl extends GlobalQuotaSettings {
     return builder.build();
   }
 
+  private boolean hasThrottle(QuotaProtos.ThrottleType quotaType,
+      QuotaProtos.Throttle.Builder throttleBuilder) {
+    boolean hasThrottle = false;
+    switch (quotaType) {
+      case REQUEST_NUMBER:
+        if (throttleBuilder.hasReqNum()) {
+          hasThrottle = true;
+        }
+        break;
+      case REQUEST_SIZE:
+        if (throttleBuilder.hasReqSize()) {
+          hasThrottle = true;
+        }
+        break;
+      case WRITE_NUMBER:
+        if (throttleBuilder.hasWriteNum()) {
+          hasThrottle = true;
+        }
+        break;
+      case WRITE_SIZE:
+        if (throttleBuilder.hasWriteSize()) {
+          hasThrottle = true;
+        }
+        break;
+      case READ_NUMBER:
+        if (throttleBuilder.hasReadNum()) {
+          hasThrottle = true;
+        }
+        break;
+      case READ_SIZE:
+        if (throttleBuilder.hasReadSize()) {
+          hasThrottle = true;
+        }
+        break;
+      case REQUEST_CAPACITY_UNIT:
+        if (throttleBuilder.hasReqCapacityUnit()) {
+          hasThrottle = true;
+        }
+        break;
+      case READ_CAPACITY_UNIT:
+        if (throttleBuilder.hasReadCapacityUnit()) {
+          hasThrottle = true;
+        }
+        break;
+      case WRITE_CAPACITY_UNIT:
+        if (throttleBuilder.hasWriteCapacityUnit()) {
+          hasThrottle = true;
+        }
+        break;
+      default:
+    }
+    return hasThrottle;
+  }
+
   @Override
   protected GlobalQuotaSettingsImpl merge(QuotaSettings other) throws IOException {
     // Validate the quota subject
@@ -124,14 +178,60 @@ public class GlobalQuotaSettingsImpl extends GlobalQuotaSettings {
       if (!otherThrottle.proto.hasType() || !otherThrottle.proto.hasTimedQuota()) {
         // It means it's a remove request
         // To prevent the "empty" row in QuotaTableUtil.QUOTA_TABLE_NAME
-        throttleBuilder = null;
+
+        QuotaProtos.ThrottleRequest otherProto = otherThrottle.proto;
+        if (throttleBuilder != null && !otherThrottle.proto.hasTimedQuota() && otherThrottle.proto
+            .hasType()) {
+          switch (otherProto.getType()) {
+            case REQUEST_NUMBER:
+              throttleBuilder.clearReqNum();
+              break;
+            case REQUEST_SIZE:
+              throttleBuilder.clearReqSize();
+              break;
+            case WRITE_NUMBER:
+              throttleBuilder.clearWriteNum();
+              break;
+            case WRITE_SIZE:
+              throttleBuilder.clearWriteSize();
+              break;
+            case READ_NUMBER:
+              throttleBuilder.clearReadNum();
+              break;
+            case READ_SIZE:
+              throttleBuilder.clearReadSize();
+              break;
+            case REQUEST_CAPACITY_UNIT:
+              throttleBuilder.clearReqCapacityUnit();
+              break;
+            case READ_CAPACITY_UNIT:
+              throttleBuilder.clearReadCapacityUnit();
+              break;
+            case WRITE_CAPACITY_UNIT:
+              throttleBuilder.clearWriteCapacityUnit();
+              break;
+            default:
+          }
+          boolean hasThrottle = false;
+          for (QuotaProtos.ThrottleType quotaType : QuotaProtos.ThrottleType.values()) {
+            hasThrottle = hasThrottle(quotaType, throttleBuilder);
+            if (hasThrottle) {
+              break;
+            }
+          }
+          if (!hasThrottle) {
+            throttleBuilder = null;
+          }
+        } else {
+          throttleBuilder = null;
+        }
+
       } else {
         QuotaProtos.ThrottleRequest otherProto = otherThrottle.proto;
         validateTimedQuota(otherProto.getTimedQuota());
         if (throttleBuilder == null) {
           throttleBuilder = QuotaProtos.Throttle.newBuilder();
         }
-
         switch (otherProto.getType()) {
           case REQUEST_NUMBER:
             throttleBuilder.setReqNum(otherProto.getTimedQuota());
@@ -166,8 +266,8 @@ public class GlobalQuotaSettingsImpl extends GlobalQuotaSettings {
     }
 
     // Propagate the space quota portion
-    QuotaProtos.SpaceQuota.Builder spaceBuilder = (spaceProto == null
-        ? null : spaceProto.toBuilder());
+    QuotaProtos.SpaceQuota.Builder spaceBuilder =
+        (spaceProto == null ? null : spaceProto.toBuilder());
     if (other instanceof SpaceLimitSettings) {
       if (spaceBuilder == null) {
         spaceBuilder = QuotaProtos.SpaceQuota.newBuilder();
@@ -181,10 +281,9 @@ public class GlobalQuotaSettingsImpl extends GlobalQuotaSettings {
         SpaceQuota quotaToMerge = spaceRequest.getQuota();
         // Validate that the two settings are for the same target.
         // SpaceQuotas either apply to a table or a namespace (no user spacequota).
-        if (!Objects.equals(getTableName(), settingsToMerge.getTableName())
-            && !Objects.equals(getNamespace(), settingsToMerge.getNamespace())) {
-          throw new IllegalArgumentException(
-              "Cannot merge " + settingsToMerge + " into " + this);
+        if (!Objects.equals(getTableName(), settingsToMerge.getTableName()) && !Objects
+            .equals(getNamespace(), settingsToMerge.getNamespace())) {
+          throw new IllegalArgumentException("Cannot merge " + settingsToMerge + " into " + this);
         }
 
         if (quotaToMerge.getRemove()) {
@@ -210,10 +309,9 @@ public class GlobalQuotaSettingsImpl extends GlobalQuotaSettings {
       return null;
     }
 
-    return new GlobalQuotaSettingsImpl(
-        getUserName(), getTableName(), getNamespace(), getRegionServer(),
-        (throttleBuilder == null ? null : throttleBuilder.build()), bypassGlobals,
-        (removeSpaceBuilder ? null : spaceBuilder.build()));
+    return new GlobalQuotaSettingsImpl(getUserName(), getTableName(), getNamespace(),
+        getRegionServer(), (throttleBuilder == null ? null : throttleBuilder.build()),
+        bypassGlobals, (removeSpaceBuilder ? null : spaceBuilder.build()));
   }
 
   private void validateTimedQuota(final TimedQuota timedQuota) throws IOException {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaThrottle.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaThrottle.java
index abc7b1d..993007df 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaThrottle.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaThrottle.java
@@ -27,7 +27,10 @@ import static org.apache.hadoop.hbase.quotas.ThrottleQuotaTestUtil.triggerUserCa
 import static org.apache.hadoop.hbase.quotas.ThrottleQuotaTestUtil.waitMinuteQuota;
 import static org.junit.Assert.assertEquals;
 
+import java.io.IOException;
+import java.util.List;
 import java.util.concurrent.TimeUnit;
+
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
@@ -71,6 +74,13 @@ public class TestQuotaThrottle {
     TableName.valueOf("TestQuotaAdmin1"),
     TableName.valueOf("TestQuotaAdmin2")
   };
+
+  private final static String[] NAMESPACES = new String[] {
+    "NAMESPACE01",
+    "NAMESPACE02",
+    "NAMESPACE03"
+  };
+
   private static Table[] tables;
 
   @BeforeClass
@@ -192,6 +202,147 @@ public class TestQuotaThrottle {
   }
 
   @Test
+  public void testUserUnThrottleByType() throws Exception {
+    final Admin admin = TEST_UTIL.getAdmin();
+    final String userName = User.getCurrent().getShortName();
+    String userName01 = "user01";
+    // Add 6req/min limit
+    admin.setQuota(QuotaSettingsFactory
+        .throttleUser(userName, ThrottleType.REQUEST_NUMBER, 6, TimeUnit.MINUTES));
+    admin.setQuota(QuotaSettingsFactory
+        .throttleUser(userName, ThrottleType.REQUEST_SIZE, 6, TimeUnit.MINUTES));
+    admin.setQuota(QuotaSettingsFactory
+        .throttleUser(userName01, ThrottleType.REQUEST_NUMBER, 6, TimeUnit.MINUTES));
+    admin.setQuota(QuotaSettingsFactory
+        .throttleUser(userName01, ThrottleType.REQUEST_SIZE, 6, TimeUnit.MINUTES));
+    admin.setQuota(
+        QuotaSettingsFactory.unthrottleUserByThrottleType(userName, ThrottleType.REQUEST_NUMBER));
+    assertEquals(3, getQuotaSettingCount(admin));
+    admin.setQuota(
+        QuotaSettingsFactory.unthrottleUserByThrottleType(userName, ThrottleType.REQUEST_SIZE));
+    assertEquals(2, getQuotaSettingCount(admin));
+  }
+
+  @Test
+  public void testUserTableUnThrottleByType() throws Exception {
+    final Admin admin = TEST_UTIL.getAdmin();
+    final String userName = User.getCurrent().getShortName();
+    String userName01 = "user01";
+    // Add 6req/min limit
+    admin.setQuota(QuotaSettingsFactory
+        .throttleUser(userName, TABLE_NAMES[0], ThrottleType.REQUEST_NUMBER, 6, TimeUnit.MINUTES));
+    admin.setQuota(QuotaSettingsFactory
+        .throttleUser(userName, TABLE_NAMES[0], ThrottleType.REQUEST_SIZE, 6, TimeUnit.MINUTES));
+    admin.setQuota(QuotaSettingsFactory
+        .throttleUser(userName01, TABLE_NAMES[1], ThrottleType.REQUEST_NUMBER, 6,
+            TimeUnit.MINUTES));
+    admin.setQuota(QuotaSettingsFactory
+        .throttleUser(userName01, TABLE_NAMES[1], ThrottleType.REQUEST_SIZE, 6, TimeUnit.MINUTES));
+    admin.setQuota(QuotaSettingsFactory
+        .unthrottleUserByThrottleType(userName, TABLE_NAMES[0], ThrottleType.REQUEST_NUMBER));
+    assertEquals(3, getQuotaSettingCount(admin));
+    admin.setQuota(QuotaSettingsFactory
+        .unthrottleUserByThrottleType(userName, TABLE_NAMES[0], ThrottleType.REQUEST_SIZE));
+    assertEquals(2, getQuotaSettingCount(admin));
+  }
+
+  @Test
+  public void testUserNameSpaceUnThrottleByType() throws Exception {
+    final Admin admin = TEST_UTIL.getAdmin();
+    final String userName = User.getCurrent().getShortName();
+    String userName01 = "user01";
+    // Add 6req/min limit
+    admin.setQuota(QuotaSettingsFactory
+        .throttleUser(userName, NAMESPACES[0], ThrottleType.REQUEST_NUMBER, 6, TimeUnit.MINUTES));
+    admin.setQuota(QuotaSettingsFactory
+        .throttleUser(userName, NAMESPACES[0], ThrottleType.REQUEST_SIZE, 6, TimeUnit.MINUTES));
+    admin.setQuota(QuotaSettingsFactory
+        .throttleUser(userName01, NAMESPACES[1], ThrottleType.REQUEST_NUMBER, 6, TimeUnit.MINUTES));
+    admin.setQuota(QuotaSettingsFactory
+        .throttleUser(userName01, NAMESPACES[1], ThrottleType.REQUEST_SIZE, 6, TimeUnit.MINUTES));
+    admin.setQuota(QuotaSettingsFactory
+        .unthrottleUserByThrottleType(userName, NAMESPACES[0], ThrottleType.REQUEST_NUMBER));
+    assertEquals(3, getQuotaSettingCount(admin));
+    admin.setQuota(QuotaSettingsFactory
+        .unthrottleUserByThrottleType(userName, NAMESPACES[0], ThrottleType.REQUEST_SIZE));
+    assertEquals(2, getQuotaSettingCount(admin));
+  }
+
+  @Test
+  public void testTableUnThrottleByType() throws Exception {
+    final Admin admin = TEST_UTIL.getAdmin();
+    final String userName = User.getCurrent().getShortName();
+    // Add 6req/min limit
+    admin.setQuota(QuotaSettingsFactory
+        .throttleTable(TABLE_NAMES[0], ThrottleType.REQUEST_NUMBER, 6, TimeUnit.MINUTES));
+    admin.setQuota(QuotaSettingsFactory
+        .throttleTable(TABLE_NAMES[0], ThrottleType.REQUEST_SIZE, 6, TimeUnit.MINUTES));
+    admin.setQuota(QuotaSettingsFactory
+        .throttleTable(TABLE_NAMES[1], ThrottleType.REQUEST_NUMBER, 6, TimeUnit.MINUTES));
+    admin.setQuota(QuotaSettingsFactory
+        .throttleTable(TABLE_NAMES[1], ThrottleType.REQUEST_SIZE, 6, TimeUnit.MINUTES));
+    admin.setQuota(QuotaSettingsFactory
+        .unthrottleTableByThrottleType(TABLE_NAMES[0], ThrottleType.REQUEST_NUMBER));
+    assertEquals(3, getQuotaSettingCount(admin));
+    admin.setQuota(QuotaSettingsFactory
+        .unthrottleTableByThrottleType(TABLE_NAMES[0], ThrottleType.REQUEST_SIZE));
+    assertEquals(2, getQuotaSettingCount(admin));
+  }
+
+  @Test
+  public void testNameSpaceUnThrottleByType() throws Exception {
+    final Admin admin = TEST_UTIL.getAdmin();
+    final String userName = User.getCurrent().getShortName();
+    // Add 6req/min limit
+    admin.setQuota(QuotaSettingsFactory
+        .throttleNamespace(NAMESPACES[0], ThrottleType.REQUEST_NUMBER, 6, TimeUnit.MINUTES));
+    admin.setQuota(QuotaSettingsFactory
+        .throttleNamespace(NAMESPACES[0], ThrottleType.REQUEST_SIZE, 6, TimeUnit.MINUTES));
+    admin.setQuota(QuotaSettingsFactory
+        .throttleNamespace(NAMESPACES[1], ThrottleType.REQUEST_NUMBER, 6, TimeUnit.MINUTES));
+    admin.setQuota(QuotaSettingsFactory
+        .throttleNamespace(NAMESPACES[1], ThrottleType.REQUEST_SIZE, 6, TimeUnit.MINUTES));
+    admin.setQuota(QuotaSettingsFactory
+        .unthrottleNamespaceByThrottleType(NAMESPACES[0], ThrottleType.REQUEST_NUMBER));
+    assertEquals(3, getQuotaSettingCount(admin));
+    admin.setQuota(QuotaSettingsFactory
+        .unthrottleNamespaceByThrottleType(NAMESPACES[0], ThrottleType.REQUEST_SIZE));
+    assertEquals(2, getQuotaSettingCount(admin));
+  }
+
+  @Test
+  public void testRegionServerUnThrottleByType() throws Exception {
+    final Admin admin = TEST_UTIL.getAdmin();
+    final String[] REGIONSERVER = { "RS01", "RS02" };
+
+    admin.setQuota(QuotaSettingsFactory
+        .throttleRegionServer(REGIONSERVER[0], ThrottleType.READ_NUMBER, 4, TimeUnit.MINUTES));
+    admin.setQuota(QuotaSettingsFactory
+        .throttleRegionServer(REGIONSERVER[0], ThrottleType.WRITE_NUMBER, 4, TimeUnit.MINUTES));
+    admin.setQuota(QuotaSettingsFactory
+        .throttleRegionServer(REGIONSERVER[1], ThrottleType.READ_NUMBER, 4, TimeUnit.MINUTES));
+    admin.setQuota(QuotaSettingsFactory
+        .throttleRegionServer(REGIONSERVER[1], ThrottleType.WRITE_NUMBER, 4, TimeUnit.MINUTES));
+
+    admin.setQuota(QuotaSettingsFactory
+        .unthrottleRegionServerByThrottleType(REGIONSERVER[0], ThrottleType.READ_NUMBER));
+    assertEquals(3, getQuotaSettingCount(admin));
+    admin.setQuota(QuotaSettingsFactory
+        .unthrottleRegionServerByThrottleType(REGIONSERVER[0], ThrottleType.WRITE_NUMBER));
+    assertEquals(2, getQuotaSettingCount(admin));
+  }
+
+  public int getQuotaSettingCount(Admin admin) throws IOException {
+    List<QuotaSettings> list_quotas = admin.getQuota(new QuotaFilter());
+    int quotaSettingCount = 0;
+    for (QuotaSettings setting : list_quotas) {
+      quotaSettingCount++;
+      LOG.info("Quota Setting:" + setting);
+    }
+    return quotaSettingCount;
+  }
+
+  @Test
   public void testUserTableReadAndWriteThrottle() throws Exception {
     final Admin admin = TEST_UTIL.getAdmin();
     final String userName = User.getCurrent().getShortName();
diff --git a/hbase-shell/src/main/ruby/hbase/quotas.rb b/hbase-shell/src/main/ruby/hbase/quotas.rb
index d4d73e9..104ed75 100644
--- a/hbase-shell/src/main/ruby/hbase/quotas.rb
+++ b/hbase-shell/src/main/ruby/hbase/quotas.rb
@@ -46,6 +46,15 @@ module HBaseQuotasConstants
   NO_WRITES = 'NO_WRITES'.freeze
   NO_WRITES_COMPACTIONS = 'NO_WRITES_COMPACTIONS'.freeze
   DISABLE = 'DISABLE'.freeze
+  READ_NUMBER = 'READ_NUMBER'.freeze
+  READ_SIZE = 'READ_SIZE'.freeze
+  WRITE_NUMBER = 'WRITE_NUMBER'.freeze
+  WRITE_SIZE = 'WRITE_SIZE'.freeze
+  REQUEST_NUMBER = 'REQUEST_NUMBER'.freeze
+  REQUEST_SIZE = 'REQUEST_SIZE'.freeze
+  REQUEST_CAPACITY_UNIT = 'REQUEST_CAPACITY_UNIT'.freeze
+  WRITE_CAPACITY_UNIT = 'WRITE_CAPACITY_UNIT'.freeze
+  READ_CAPACITY_UNIT = 'READ_CAPACITY_UNIT'.freeze
 end
 
 module Hbase
@@ -102,39 +111,124 @@ module Hbase
 
     def unthrottle(args)
       raise(ArgumentError, 'Arguments should be a Hash') unless args.is_a?(Hash)
-      if args.key?(USER)
-        user = args.delete(USER)
-        if args.key?(TABLE)
-          table = TableName.valueOf(args.delete(TABLE))
-          raise(ArgumentError, 'Unexpected arguments: ' + args.inspect) unless args.empty?
-          settings = QuotaSettingsFactory.unthrottleUser(user, table)
-        elsif args.key?(NAMESPACE)
-          namespace = args.delete(NAMESPACE)
-          raise(ArgumentError, 'Unexpected arguments: ' + args.inspect) unless args.empty?
-          settings = QuotaSettingsFactory.unthrottleUser(user, namespace)
-        else
-          raise(ArgumentError, 'Unexpected arguments: ' + args.inspect) unless args.empty?
-          settings = QuotaSettingsFactory.unthrottleUser(user)
-        end
-      elsif args.key?(TABLE)
-        table = TableName.valueOf(args.delete(TABLE))
-        raise(ArgumentError, 'Unexpected arguments: ' + args.inspect) unless args.empty?
-        settings = QuotaSettingsFactory.unthrottleTable(table)
-      elsif args.key?(NAMESPACE)
-        namespace = args.delete(NAMESPACE)
-        raise(ArgumentError, 'Unexpected arguments: ' + args.inspect) unless args.empty?
-        settings = QuotaSettingsFactory.unthrottleNamespace(namespace)
+
+      if args.key?(USER) then settings = unthrottle_user_table_namespace(args)
+      elsif args.key?(TABLE) then settings = unthrottle_table(args)
+      elsif args.key?(NAMESPACE) then settings = unthrottle_namespace(args)
       elsif args.key?(REGIONSERVER)
-        regionServer = args.delete(REGIONSERVER)
-        raise(ArgumentError, 'Unexpected arguments: ' + args.inspect) unless args.empty?
-        # TODO: Setting specified region server quota isn't supported currently and using 'all' for all RS
-        settings = QuotaSettingsFactory.unthrottleRegionServer('all')
+        settings = unthrottle_regionserver(args)
       else
         raise 'One of USER, TABLE, NAMESPACE or REGIONSERVER must be specified'
       end
       @admin.setQuota(settings)
     end
 
+    def _parse_throttle_type(type_cls, throttle_type)
+      type_cls.valueOf(throttle_type)
+    end
+
+    def get_throttle_type(args)
+      throttle_type_str = args.delete(THROTTLE_TYPE)
+      throttle_type = _parse_throttle_type(ThrottleType, throttle_type_str)
+      throttle_type
+    end
+
+    def unthrottle_user_table_namespace(args)
+      user = args.delete(USER)
+      settings = if args.key?(TABLE)
+                   unthrottle_user_table(args, user)
+                 elsif args.key?(NAMESPACE)
+                   unthrottle_user_namespace(args, user)
+                 else
+                   unthrottle_user(args, user)
+                 end
+      settings
+    end
+
+    def args_empty(args)
+      return if args.empty?
+
+      raise(ArgumentError,
+            'Unexpected arguments: ' + args.inspect)
+    end
+
+    def unthrottle_user_table(args, user)
+      table = TableName.valueOf(args.delete(TABLE))
+      if args.key?(THROTTLE_TYPE)
+        settings = QuotaSettingsFactory
+                   .unthrottleUserByThrottleType(user,
+                                                 table, get_throttle_type(args))
+      else
+        args_empty(args)
+        settings = QuotaSettingsFactory.unthrottleUser(user, table)
+      end
+      settings
+    end
+
+    def unthrottle_user_namespace(args, user)
+      namespace = args.delete(NAMESPACE)
+      if args.key?(THROTTLE_TYPE)
+        throttle_type = get_throttle_type(args)
+        settings = QuotaSettingsFactory
+                   .unthrottleUserByThrottleType(user, namespace, throttle_type)
+      else
+        args_empty(args)
+        settings = QuotaSettingsFactory.unthrottleUser(user, namespace)
+      end
+      settings
+    end
+
+    def unthrottle_user(args, user)
+      if args.key?(THROTTLE_TYPE)
+        throttle_type = get_throttle_type(args)
+        settings = QuotaSettingsFactory
+                   .unthrottleUserByThrottleType(user, throttle_type)
+      else
+        args_empty(args)
+        settings = QuotaSettingsFactory.unthrottleUser(user)
+      end
+      settings
+    end
+
+    def unthrottle_table(args)
+      table = TableName.valueOf(args.delete(TABLE))
+      if args.key?(THROTTLE_TYPE)
+        throttle_type = get_throttle_type(args)
+        settings = QuotaSettingsFactory
+                   .unthrottleTableByThrottleType(table, throttle_type)
+      else
+        args_empty(args)
+        settings = QuotaSettingsFactory.unthrottleTable(table)
+      end
+      settings
+    end
+
+    def unthrottle_namespace(args)
+      namespace = args.delete(NAMESPACE)
+      if args.key?(THROTTLE_TYPE)
+        throttle_type = get_throttle_type(args)
+        settings = QuotaSettingsFactory
+                   .unthrottleNamespaceByThrottleType(namespace, throttle_type)
+      else
+        args_empty(args)
+        settings = QuotaSettingsFactory.unthrottleNamespace(namespace)
+      end
+      settings
+    end
+
+    def unthrottle_regionserver(args)
+      _region_server = args.delete(REGIONSERVER)
+      if args.key?(THROTTLE_TYPE)
+        throttle_type = get_throttle_type(args)
+        settings = QuotaSettingsFactory
+                   .unthrottleRegionServerByThrottleType('all', throttle_type)
+      else
+        args_empty(args)
+        settings = QuotaSettingsFactory.unthrottleRegionServer('all')
+      end
+      settings
+    end
+
     # rubocop:disable Metrics/AbcSize, Metrics/CyclomaticComplexity
     # rubocop:disable Metrics/MethodLength, Metrics/PerceivedComplexity
     def limit_space(args)


[hbase] 28/49: HBASE-22093 Combine TestRestoreSnapshotFromClientWithRegionReplicas to CloneSnapshotFromClientAfterSplittingRegionTestBase#testCloneSnapshotAfterSplittingRegion

Posted by op...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

openinx pushed a commit to branch HBASE-21879
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit 90a5db4e31b5db598a93513f3acfe61f88aa9a6b
Author: Toshihiro Suzuki <br...@gmail.com>
AuthorDate: Sat Mar 23 15:24:33 2019 +0900

    HBASE-22093 Combine TestRestoreSnapshotFromClientWithRegionReplicas to CloneSnapshotFromClientAfterSplittingRegionTestBase#testCloneSnapshotAfterSplittingRegion
    
    Signed-off-by: Duo Zhang <zh...@apache.org>
---
 ...shotFromClientAfterSplittingRegionTestBase.java |  2 +
 ...estoreSnapshotFromClientWithRegionReplicas.java | 62 ----------------------
 2 files changed, 2 insertions(+), 62 deletions(-)

diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/CloneSnapshotFromClientAfterSplittingRegionTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/CloneSnapshotFromClientAfterSplittingRegionTestBase.java
index d1108c0..5ed100f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/CloneSnapshotFromClientAfterSplittingRegionTestBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/CloneSnapshotFromClientAfterSplittingRegionTestBase.java
@@ -57,6 +57,8 @@ public class CloneSnapshotFromClientAfterSplittingRegionTestBase
       admin.cloneSnapshot(snapshotName2, clonedTableName);
       SnapshotTestingUtils.waitForTableToBeOnline(TEST_UTIL, clonedTableName);
 
+      verifyRowCount(TEST_UTIL, clonedTableName, snapshot1Rows);
+
       RegionStates regionStates =
         TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager().getRegionStates();
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClientWithRegionReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClientWithRegionReplicas.java
deleted file mode 100644
index ea47e9d..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClientWithRegionReplicas.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.client;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.hadoop.hbase.HBaseClassTestRule;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.testclassification.ClientTests;
-import org.apache.hadoop.hbase.testclassification.LargeTests;
-import org.junit.ClassRule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category({ LargeTests.class, ClientTests.class })
-public class TestRestoreSnapshotFromClientWithRegionReplicas
-    extends RestoreSnapshotFromClientTestBase {
-
-  @ClassRule
-  public static final HBaseClassTestRule CLASS_RULE =
-    HBaseClassTestRule.forClass(TestRestoreSnapshotFromClientWithRegionReplicas.class);
-
-  @Override
-  protected int getNumReplicas() {
-    return 3;
-  }
-
-  @Test
-  public void testOnlineSnapshotAfterSplittingRegions() throws IOException, InterruptedException {
-    List<RegionInfo> regionInfos = admin.getRegions(tableName);
-    RegionReplicaUtil.removeNonDefaultRegions(regionInfos);
-
-    // Split a region
-    splitRegion(regionInfos.get(0));
-
-    // Take a online snapshot
-    admin.snapshot(snapshotName1, tableName);
-
-    // Clone the snapshot to another table
-    TableName clonedTableName =
-      TableName.valueOf(name.getMethodName() + "-" + System.currentTimeMillis());
-    admin.cloneSnapshot(snapshotName1, clonedTableName);
-
-    verifyRowCount(TEST_UTIL, clonedTableName, snapshot1Rows);
-  }
-}


[hbase] 29/49: HBASE-22052 pom cleaning; filter out jersey-core in hadoop2 to match hadoop3 and remove redunant version specifications

Posted by op...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

openinx pushed a commit to branch HBASE-21879
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit 227ca4b3cf446899922bac83710321faae640311
Author: stack <st...@apache.org>
AuthorDate: Mon Mar 25 09:25:15 2019 -0400

    HBASE-22052 pom cleaning; filter out jersey-core in hadoop2 to match hadoop3 and remove redunant version specifications
    
    This is a reapply of a reverted commit. This commit includes
    HBASE-22059 amendment and subsequent ammendments to HBASE-22052.
    See HBASE-22052 for full story.
    
    jersey-core is problematic. It was transitively included from hadoop
    and polluting our CLASSPATH with an implementation of a 1.x version
    of the javax.ws.rs.core.Response Interface from jsr311-api when we
    want the javax.ws.rs-api 2.x version.
    
        M hbase-endpoint/pom.xml
        M hbase-http/pom.xml
        M hbase-mapreduce/pom.xml
        M hbase-rest/pom.xml
        M hbase-server/pom.xml
        M hbase-zookeeper/pom.xml
         Remove redundant version specification (and the odd property define
         done already up in parent pom).
        M hbase-it/pom.xml
        M hbase-rest/pom.xml
         Exclude jersey-core explicitly.
    
        M hbase-procedure/pom.xml
         Remove redundant version and classifier.
    
        M pom.xml
         Add jersey-core exclusions to all dependencies that pull it in
         except hadoop-minicluster. mr tests fail w/o the jersey-core
         so let it in for minicluster and then in modules, exclude it
         where it causes damage as in hbase-it.
---
 hbase-endpoint/pom.xml  |  3 --
 hbase-http/pom.xml      |  3 --
 hbase-it/pom.xml        | 24 +++++++++++---
 hbase-mapreduce/pom.xml |  3 --
 hbase-rest/pom.xml      | 28 ++++++++++------
 hbase-server/pom.xml    | 26 +++++++++++++++
 hbase-zookeeper/pom.xml |  3 --
 pom.xml                 | 86 +++++++++++++++++++++++++++++++++++++++++++++++++
 8 files changed, 149 insertions(+), 27 deletions(-)

diff --git a/hbase-endpoint/pom.xml b/hbase-endpoint/pom.xml
index d0b5425..334e0ad 100644
--- a/hbase-endpoint/pom.xml
+++ b/hbase-endpoint/pom.xml
@@ -327,9 +327,6 @@
           <value>3.0</value>
         </property>
       </activation>
-      <properties>
-        <hadoop.version>${hadoop-three.version}</hadoop.version>
-      </properties>
       <dependencies>
         <dependency>
           <groupId>org.apache.hadoop</groupId>
diff --git a/hbase-http/pom.xml b/hbase-http/pom.xml
index fb70451..c32c581 100644
--- a/hbase-http/pom.xml
+++ b/hbase-http/pom.xml
@@ -416,9 +416,6 @@
           <value>3.0</value>
         </property>
       </activation>
-      <properties>
-        <hadoop.version>${hadoop-three.version}</hadoop.version>
-      </properties>
       <dependencies>
         <dependency>
           <groupId>org.apache.hadoop</groupId>
diff --git a/hbase-it/pom.xml b/hbase-it/pom.xml
index 956b87e..d24695f 100644
--- a/hbase-it/pom.xml
+++ b/hbase-it/pom.xml
@@ -162,6 +162,16 @@
   </build>
 
   <dependencies>
+    <!--This one is upfront to get in front of
+         any dependency that pulls in jersey-core.
+         Jersey-core has implemented version 1
+         Interfaces of what is in this dependency
+         which does version 2.-->
+    <dependency>
+      <groupId>javax.ws.rs</groupId>
+      <artifactId>javax.ws.rs-api</artifactId>
+      <scope>test</scope>
+    </dependency>
 	<!-- Intra-project dependencies -->
     <dependency>
       <groupId>org.apache.hbase</groupId>
@@ -218,6 +228,15 @@
     <dependency>
       <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-testing-util</artifactId>
+      <exclusions>
+        <!--This dependency pulls in hadoop-minicluster
+             which pulls in the below. It messes up
+             this build at assembly time. See HBASE-22029-->
+        <exclusion>
+         <groupId>com.sun.jersey</groupId>
+         <artifactId>jersey-core</artifactId>
+        </exclusion>
+      </exclusions>
     </dependency>
     <dependency>
       <groupId>org.apache.hbase</groupId>
@@ -248,11 +267,6 @@
       <artifactId>htrace-core4</artifactId>
     </dependency>
     <dependency>
-      <groupId>javax.ws.rs</groupId>
-      <artifactId>javax.ws.rs-api</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
       <groupId>org.glassfish.jersey.core</groupId>
       <artifactId>jersey-client</artifactId>
     </dependency>
diff --git a/hbase-mapreduce/pom.xml b/hbase-mapreduce/pom.xml
index 19c2e4f..f2ef386 100644
--- a/hbase-mapreduce/pom.xml
+++ b/hbase-mapreduce/pom.xml
@@ -401,9 +401,6 @@
           <value>3.0</value>
         </property>
       </activation>
-      <properties>
-        <hadoop.version>${hadoop-three.version}</hadoop.version>
-      </properties>
       <dependencies>
         <dependency>
           <groupId>org.apache.hadoop</groupId>
diff --git a/hbase-rest/pom.xml b/hbase-rest/pom.xml
index 8d9e537..05f0574 100644
--- a/hbase-rest/pom.xml
+++ b/hbase-rest/pom.xml
@@ -172,6 +172,15 @@
     </pluginManagement>
   </build>
   <dependencies>
+    <!--This one is upfront to get in front of
+         any dependency that pulls in jersey-core.
+         Jersey-core has implemented version 1
+         Interfaces of what is in this dependency
+         which does version 2.-->
+    <dependency>
+      <groupId>javax.ws.rs</groupId>
+      <artifactId>javax.ws.rs-api</artifactId>
+    </dependency>
     <!-- Intra-project dependencies -->
     <dependency>
       <groupId>org.apache.hbase</groupId>
@@ -227,6 +236,15 @@
       <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-testing-util</artifactId>
       <scope>test</scope>
+      <exclusions>
+        <!--This dependency pulls in hadoop-minicluster
+             which pulls in the below. It messes up
+             this build at assembly time. See HBASE-22029-->
+        <exclusion>
+         <groupId>com.sun.jersey</groupId>
+         <artifactId>jersey-core</artifactId>
+        </exclusion>
+      </exclusions>
     </dependency>
     <dependency>
       <groupId>org.apache.hbase</groupId>
@@ -274,10 +292,6 @@
       <artifactId>javax.servlet-api</artifactId>
     </dependency>
     <dependency>
-      <groupId>javax.ws.rs</groupId>
-      <artifactId>javax.ws.rs-api</artifactId>
-    </dependency>
-    <dependency>
       <groupId>org.eclipse.jetty</groupId>
       <artifactId>jetty-server</artifactId>
     </dependency>
@@ -426,7 +440,6 @@
         <dependency>
           <groupId>org.apache.hadoop</groupId>
           <artifactId>hadoop-annotations</artifactId>
-          <version>${hadoop-two.version}</version>
         </dependency>
         <dependency>
           <groupId>org.apache.hadoop</groupId>
@@ -455,7 +468,6 @@
           <dependency>
             <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-yarn-server-nodemanager</artifactId>
-            <version>${hadoop-three.version}</version>
             <exclusions>
               <exclusion>
                 <groupId>com.sun.jersey</groupId>
@@ -466,7 +478,6 @@
           <dependency>
             <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-yarn-server-resourcemanager</artifactId>
-            <version>${hadoop-three.version}</version>
             <exclusions>
               <exclusion>
                 <groupId>com.sun.jersey</groupId>
@@ -477,7 +488,6 @@
           <dependency>
             <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-yarn-server-timelineservice</artifactId>
-            <version>${hadoop-three.version}</version>
             <exclusions>
               <exclusion>
                 <groupId>javax.ws.rs</groupId>
@@ -488,7 +498,6 @@
           <dependency>
             <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-yarn-common</artifactId>
-            <version>${hadoop-three.version}</version>
             <exclusions>
               <exclusion>
                 <groupId>com.sun.jersey</groupId>
@@ -507,7 +516,6 @@
         <dependency>
           <groupId>org.apache.hadoop</groupId>
           <artifactId>hadoop-annotations</artifactId>
-          <version>${hadoop-three.version}</version>
         </dependency>
         <dependency>
           <groupId>org.apache.hadoop</groupId>
diff --git a/hbase-server/pom.xml b/hbase-server/pom.xml
index d101e89..725890c 100644
--- a/hbase-server/pom.xml
+++ b/hbase-server/pom.xml
@@ -678,10 +678,25 @@
       <dependencies>
         <dependency>
           <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-distcp</artifactId>
+          <!--This dependency is not in top-level pom so needs
+               version specified-->
+          <version>${hadoop-two.version}</version>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
           <artifactId>hadoop-common</artifactId>
         </dependency>
         <dependency>
           <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-auth</artifactId>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-annotations</artifactId>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
           <artifactId>hadoop-mapreduce-client-core</artifactId>
           <exclusions>
             <exclusion>
@@ -774,6 +789,13 @@
       <dependencies>
         <dependency>
           <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-distcp</artifactId>
+          <!--This dependency is not in top-level pom so needs
+               version specified-->
+          <version>${hadoop-three.version}</version>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
           <artifactId>hadoop-common</artifactId>
         </dependency>
         <dependency>
@@ -803,6 +825,10 @@
         </dependency>
         <dependency>
           <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-annotations</artifactId>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
           <artifactId>hadoop-minicluster</artifactId>
           <scope>test</scope>
         </dependency>
diff --git a/hbase-zookeeper/pom.xml b/hbase-zookeeper/pom.xml
index b96c717..4504795 100644
--- a/hbase-zookeeper/pom.xml
+++ b/hbase-zookeeper/pom.xml
@@ -342,9 +342,6 @@
           <value>3.0</value>
         </property>
       </activation>
-      <properties>
-        <hadoop.version>${hadoop-three.version}</hadoop.version>
-      </properties>
       <dependencies>
         <dependency>
           <groupId>org.apache.hadoop</groupId>
diff --git a/pom.xml b/pom.xml
index d3b9fd3..8c0745c 100755
--- a/pom.xml
+++ b/pom.xml
@@ -2577,6 +2577,10 @@
             <version>${hadoop-two.version}</version>
             <exclusions>
               <exclusion>
+                <groupId>com.sun.jersey</groupId>
+                <artifactId>jersey-core</artifactId>
+              </exclusion>
+              <exclusion>
                 <groupId>io.netty</groupId>
                 <artifactId>netty</artifactId>
               </exclusion>
@@ -2604,6 +2608,10 @@
             <version>${hadoop-two.version}</version>
             <exclusions>
               <exclusion>
+                <groupId>com.sun.jersey</groupId>
+                <artifactId>jersey-core</artifactId>
+              </exclusion>
+              <exclusion>
                 <groupId>io.netty</groupId>
                 <artifactId>netty</artifactId>
               </exclusion>
@@ -2621,6 +2629,10 @@
             <scope>test</scope>
             <exclusions>
               <exclusion>
+                <groupId>com.sun.jersey</groupId>
+                <artifactId>jersey-core</artifactId>
+              </exclusion>
+              <exclusion>
                 <groupId>io.netty</groupId>
                 <artifactId>netty</artifactId>
               </exclusion>
@@ -2636,6 +2648,10 @@
             <version>${hadoop-two.version}</version>
             <exclusions>
               <exclusion>
+                <groupId>com.sun.jersey</groupId>
+                <artifactId>jersey-core</artifactId>
+              </exclusion>
+              <exclusion>
                 <groupId>javax.servlet.jsp</groupId>
                 <artifactId>jsp-api</artifactId>
               </exclusion>
@@ -2708,6 +2724,10 @@
                 <groupId>com.google.guava</groupId>
                 <artifactId>guava</artifactId>
               </exclusion>
+              <exclusion>
+                <groupId>com.sun.jersey</groupId>
+                <artifactId>jersey-core</artifactId>
+              </exclusion>
             </exclusions>
           </dependency>
           <dependency>
@@ -2721,6 +2741,10 @@
             <version>${hadoop-two.version}</version>
             <exclusions>
               <exclusion>
+                <groupId>com.sun.jersey</groupId>
+                <artifactId>jersey-core</artifactId>
+              </exclusion>
+              <exclusion>
                 <groupId>commons-beanutils</groupId>
                 <artifactId>commons-beanutils</artifactId>
               </exclusion>
@@ -2760,6 +2784,23 @@
           </dependency>
           <dependency>
             <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-common</artifactId>
+            <version>${hadoop-two.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+            <exclusions>
+              <exclusion>
+                <groupId>com.sun.jersey</groupId>
+                <artifactId>jersey-core</artifactId>
+              </exclusion>
+              <exclusion>
+                <groupId>com.google.code.findbugs</groupId>
+                <artifactId>jsr305</artifactId>
+              </exclusion>
+            </exclusions>
+          </dependency>
+          <dependency>
+            <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-client</artifactId>
             <version>${hadoop-two.version}</version>
           </dependency>
@@ -2770,6 +2811,18 @@
             <artifactId>hadoop-minicluster</artifactId>
             <version>${hadoop-two.version}</version>
             <exclusions>
+              <!--If we comment this in, a few tests in hbase-mapreduce
+                   fail. They depend on jersey-core somehow. But excluding
+                   jersey-core here messes up hbase-it because jersey-core
+                   implements a 1.x jaxrs Response Interface when we depend
+                   on the 2.x Interface... . Letting this jar come in
+                   transitively here but will exclude it down in hbase-it.
+                   See HBASE-22029.
+              <exclusion>
+                <groupId>com.sun.jersey</groupId>
+                <artifactId>jersey-core</artifactId>
+              </exclusion>
+              -->
               <exclusion>
                 <groupId>commons-httpclient</groupId>
                 <artifactId>commons-httpclient</artifactId>
@@ -2802,6 +2855,11 @@
             <version>${hadoop-two.version}</version>
             <scope>test</scope>
           </dependency>
+         <dependency>
+           <groupId>org.apache.hadoop</groupId>
+           <artifactId>hadoop-annotations</artifactId>
+           <version>${hadoop-two.version}</version>
+         </dependency>
         </dependencies>
       </dependencyManagement>
     </profile>
@@ -2838,6 +2896,10 @@
            <version>${hadoop-three.version}</version>
            <exclusions>
              <exclusion>
+               <groupId>com.sun.jersey</groupId>
+               <artifactId>jersey-core</artifactId>
+             </exclusion>
+             <exclusion>
                <groupId>io.netty</groupId>
                <artifactId>netty</artifactId>
              </exclusion>
@@ -2865,6 +2927,10 @@
            <version>${hadoop-three.version}</version>
            <exclusions>
              <exclusion>
+               <groupId>com.sun.jersey</groupId>
+               <artifactId>jersey-core</artifactId>
+             </exclusion>
+             <exclusion>
                <groupId>io.netty</groupId>
                <artifactId>netty</artifactId>
              </exclusion>
@@ -2878,6 +2944,10 @@
            <scope>test</scope>
            <exclusions>
              <exclusion>
+               <groupId>com.sun.jersey</groupId>
+               <artifactId>jersey-core</artifactId>
+             </exclusion>
+             <exclusion>
                <groupId>io.netty</groupId>
                <artifactId>netty</artifactId>
              </exclusion>
@@ -3062,6 +3132,10 @@
                <groupId>com.sun.jersey</groupId>
                <artifactId>jersey-core</artifactId>
              </exclusion>
+             <exclusion>
+               <groupId>com.google.code.findbugs</groupId>
+               <artifactId>jsr305</artifactId>
+             </exclusion>
            </exclusions>
          </dependency>
          <dependency>
@@ -3081,6 +3155,18 @@
            <artifactId>hadoop-minicluster</artifactId>
            <version>${hadoop-three.version}</version>
            <exclusions>
+              <!--If we comment this in, a few tests in hbase-mapreduce
+                   fail. They depend on jersey-core somehow. But excluding
+                   jersey-core here messes up hbase-it because jersey-core
+                   implements a 1.x jaxrs Response Interface when we depend
+                   on the 2.x Interface... . Letting this jar come in
+                   transitively here but will exclude it down in hbase-it.
+                   See HBASE-22029.
+             <exclusion>
+               <groupId>com.sun.jersey</groupId>
+               <artifactId>jersey-core</artifactId>
+             </exclusion>
+             -->
              <exclusion>
                <groupId>commons-httpclient</groupId>
                <artifactId>commons-httpclient</artifactId>


[hbase] 07/49: HBASE-20662 Increasing space quota on a violated table does not remove SpaceViolationPolicy.DISABLE enforcement

Posted by op...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

openinx pushed a commit to branch HBASE-21879
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit c37b961e96433435152a542793fa7446dc43664a
Author: Nihal Jain <ni...@gmail.com>
AuthorDate: Mon Feb 4 18:13:32 2019 +0530

    HBASE-20662 Increasing space quota on a violated table does not remove SpaceViolationPolicy.DISABLE enforcement
    
    Signed-off-by: Josh Elser <el...@apache.org>
---
 .../apache/hadoop/hbase/quotas/QuotaTableUtil.java |  36 ++++++-
 .../org/apache/hadoop/hbase/master/HMaster.java    |  16 +--
 .../hadoop/hbase/quotas/MasterQuotaManager.java    |  10 ++
 .../hadoop/hbase/quotas/QuotaObserverChore.java    |  53 ++++++++--
 .../org/apache/hadoop/hbase/quotas/QuotaUtil.java  |  34 ++++++
 .../hbase/quotas/SpaceQuotaRefresherChore.java     |  16 ++-
 .../DisableTableViolationPolicyEnforcement.java    |  38 ++-----
 .../hadoop/hbase/quotas/TestSpaceQuotas.java       | 115 +++++++++++++++++++--
 8 files changed, 257 insertions(+), 61 deletions(-)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaTableUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaTableUtil.java
index b932242..2b5cb02 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaTableUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/quotas/QuotaTableUtil.java
@@ -291,6 +291,18 @@ public class QuotaTableUtil {
   }
 
   /**
+   * Creates a {@link Get} which returns only {@link SpaceQuotaSnapshot} from the quota table for a
+   * specific table.
+   * @param tn table name to get from. Can't be null.
+   */
+  public static Get makeQuotaSnapshotGetForTable(TableName tn) {
+    Get g = new Get(getTableRowKey(tn));
+    // Limit to "u:v" column
+    g.addColumn(QUOTA_FAMILY_USAGE, QUOTA_QUALIFIER_POLICY);
+    return g;
+  }
+
+  /**
    * Extracts the {@link SpaceViolationPolicy} and {@link TableName} from the provided
    * {@link Result} and adds them to the given {@link Map}. If the result does not contain
    * the expected information or the serialized policy in the value is invalid, this method
@@ -302,7 +314,7 @@ public class QuotaTableUtil {
   public static void extractQuotaSnapshot(
       Result result, Map<TableName,SpaceQuotaSnapshot> snapshots) {
     byte[] row = Objects.requireNonNull(result).getRow();
-    if (row == null) {
+    if (row == null || row.length == 0) {
       throw new IllegalArgumentException("Provided result had a null row");
     }
     final TableName targetTableName = getTableFromRowKey(row);
@@ -609,6 +621,28 @@ public class QuotaTableUtil {
     }
   }
 
+  /**
+   * Returns the current space quota snapshot of the given {@code tableName} from
+   * {@code QuotaTableUtil.QUOTA_TABLE_NAME} or null if the no quota information is available for
+   * that tableName.
+   * @param conn connection to re-use
+   * @param tableName name of the table whose current snapshot is to be retreived
+   */
+  public static SpaceQuotaSnapshot getCurrentSnapshotFromQuotaTable(Connection conn,
+      TableName tableName) throws IOException {
+    try (Table quotaTable = conn.getTable(QuotaTableUtil.QUOTA_TABLE_NAME)) {
+      Map<TableName, SpaceQuotaSnapshot> snapshots = new HashMap<>(1);
+      Result result = quotaTable.get(makeQuotaSnapshotGetForTable(tableName));
+      // if we don't have any row corresponding to this get, return null
+      if (result.isEmpty()) {
+        return null;
+      }
+      // otherwise, extract quota snapshot in snapshots object
+      extractQuotaSnapshot(result, snapshots);
+      return snapshots.get(tableName);
+    }
+  }
+
   /* =========================================================================
    *  Quotas protobuf helpers
    */
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index 10bfade..a5961da 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -166,10 +166,14 @@ import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore;
 import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
 import org.apache.hadoop.hbase.quotas.MasterQuotasObserver;
 import org.apache.hadoop.hbase.quotas.QuotaObserverChore;
+import org.apache.hadoop.hbase.quotas.QuotaTableUtil;
 import org.apache.hadoop.hbase.quotas.QuotaUtil;
 import org.apache.hadoop.hbase.quotas.SnapshotQuotaObserverChore;
+import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot;
+import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus;
 import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshotNotifier;
 import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshotNotifierFactory;
+import org.apache.hadoop.hbase.quotas.SpaceViolationPolicy;
 import org.apache.hadoop.hbase.regionserver.DefaultStoreEngine;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.HStore;
@@ -228,8 +232,6 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
 
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceViolationPolicy;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
 
@@ -2579,10 +2581,12 @@ public class HMaster extends HRegionServer implements MasterServices {
         MasterQuotaManager quotaManager = getMasterQuotaManager();
         if (quotaManager != null) {
           if (quotaManager.isQuotaInitialized()) {
-            Quotas quotaForTable = QuotaUtil.getTableQuota(getConnection(), tableName);
-            if (quotaForTable != null && quotaForTable.hasSpace()) {
-              SpaceViolationPolicy policy = quotaForTable.getSpace().getViolationPolicy();
-              if (SpaceViolationPolicy.DISABLE == policy) {
+              SpaceQuotaSnapshot currSnapshotOfTable =
+                  QuotaTableUtil.getCurrentSnapshotFromQuotaTable(getConnection(), tableName);
+              if (currSnapshotOfTable != null) {
+                SpaceQuotaStatus quotaStatus = currSnapshotOfTable.getQuotaStatus();
+                if (quotaStatus.isInViolation()
+                    && SpaceViolationPolicy.DISABLE == quotaStatus.getPolicy().orElse(null)) {
                 throw new AccessDeniedException("Enabling the table '" + tableName
                     + "' is disallowed due to a violated space quota.");
               }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
index ffef30e..65a47d1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
@@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
 import org.apache.hadoop.hbase.master.procedure.SwitchRpcThrottleProcedure;
 import org.apache.hadoop.hbase.namespace.NamespaceAuditor;
+import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.yetus.audience.InterfaceStability;
@@ -279,7 +280,16 @@ public class MasterQuotaManager implements RegionStateListener {
       }
       @Override
       public void delete() throws IOException {
+        SpaceQuotaSnapshot currSnapshotOfTable =
+            QuotaTableUtil.getCurrentSnapshotFromQuotaTable(masterServices.getConnection(), table);
         QuotaUtil.deleteTableQuota(masterServices.getConnection(), table);
+        if (currSnapshotOfTable != null) {
+          SpaceQuotaStatus quotaStatus = currSnapshotOfTable.getQuotaStatus();
+          if (SpaceViolationPolicy.DISABLE == quotaStatus.getPolicy().orElse(null)
+              && quotaStatus.isInViolation()) {
+            QuotaUtil.enableTableIfNotEnabled(masterServices.getConnection(), table);
+          }
+        }
       }
       @Override
       public void preApply(GlobalQuotaSettingsImpl quotaPojo) throws IOException {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaObserverChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaObserverChore.java
index 869ead3..92a149c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaObserverChore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaObserverChore.java
@@ -188,7 +188,8 @@ public class QuotaObserverChore extends ScheduledChore {
 
     for (TableName tableInLimbo : tablesInLimbo) {
       final SpaceQuotaSnapshot currentSnapshot = tableSnapshotStore.getCurrentState(tableInLimbo);
-      if (currentSnapshot.getQuotaStatus().isInViolation()) {
+      SpaceQuotaStatus currentStatus = currentSnapshot.getQuotaStatus();
+      if (currentStatus.isInViolation()) {
         if (LOG.isTraceEnabled()) {
           LOG.trace("Moving " + tableInLimbo + " out of violation because fewer region sizes were"
               + " reported than required.");
@@ -199,6 +200,10 @@ public class QuotaObserverChore extends ScheduledChore {
         this.snapshotNotifier.transitionTable(tableInLimbo, targetSnapshot);
         // Update it in the Table QuotaStore so that memory is consistent with no violation.
         tableSnapshotStore.setCurrentState(tableInLimbo, targetSnapshot);
+        // In case of Disable SVP, we need to enable the table as it moves out of violation
+        if (SpaceViolationPolicy.DISABLE == currentStatus.getPolicy().orElse(null)) {
+          QuotaUtil.enableTableIfNotEnabled(conn, tableInLimbo);
+        }
       }
     }
 
@@ -324,20 +329,35 @@ public class QuotaObserverChore extends ScheduledChore {
 
     // If we're changing something, log it.
     if (!currentSnapshot.equals(targetSnapshot)) {
+      this.snapshotNotifier.transitionTable(table, targetSnapshot);
+      // Update it in memory
+      tableSnapshotStore.setCurrentState(table, targetSnapshot);
+
       // If the target is none, we're moving out of violation. Update the hbase:quota table
+      SpaceViolationPolicy currPolicy = currentStatus.getPolicy().orElse(null);
+      SpaceViolationPolicy targetPolicy = targetStatus.getPolicy().orElse(null);
       if (!targetStatus.isInViolation()) {
+        // In case of Disable SVP, we need to enable the table as it moves out of violation
+        if (isDisableSpaceViolationPolicy(currPolicy, targetPolicy)) {
+          QuotaUtil.enableTableIfNotEnabled(conn, table);
+        }
         if (LOG.isDebugEnabled()) {
-          LOG.debug(table + " moving into observance of table space quota.");
+          LOG.debug(table + " moved into observance of table space quota.");
         }
-      } else if (LOG.isDebugEnabled()) {
+      } else {
         // We're either moving into violation or changing violation policies
-        LOG.debug(table + " moving into violation of table space quota with policy of "
-            + targetStatus.getPolicy());
+        if (currPolicy != targetPolicy && SpaceViolationPolicy.DISABLE == currPolicy) {
+          // In case of policy switch, we need to enable the table if current policy is Disable SVP
+          QuotaUtil.enableTableIfNotEnabled(conn, table);
+        } else if (SpaceViolationPolicy.DISABLE == targetPolicy) {
+          // In case of Disable SVP, we need to disable the table as it moves into violation
+          QuotaUtil.disableTableIfNotDisabled(conn, table);
+        }
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(
+            table + " moved into violation of table space quota with policy of " + targetPolicy);
+        }
       }
-
-      this.snapshotNotifier.transitionTable(table, targetSnapshot);
-      // Update it in memory
-      tableSnapshotStore.setCurrentState(table, targetSnapshot);
     } else if (LOG.isTraceEnabled()) {
       // Policies are the same, so we have nothing to do except log this. Don't need to re-update
       // the quota table
@@ -350,6 +370,19 @@ public class QuotaObserverChore extends ScheduledChore {
   }
 
   /**
+   * Method to check whether we are dealing with DISABLE {@link SpaceViolationPolicy}. In such a
+   * case, currPolicy or/and targetPolicy will be having DISABLE policy.
+   * @param currPolicy currently set space violation policy
+   * @param targetPolicy new space violation policy
+   * @return true if is DISABLE space violation policy; otherwise false
+   */
+  private boolean isDisableSpaceViolationPolicy(final SpaceViolationPolicy currPolicy,
+      final SpaceViolationPolicy targetPolicy) {
+    return SpaceViolationPolicy.DISABLE == currPolicy
+        || SpaceViolationPolicy.DISABLE == targetPolicy;
+  }
+
+  /**
    * Updates the hbase:quota table with the target quota policy for this <code>namespace</code>
    * if necessary.
    *
@@ -363,7 +396,7 @@ public class QuotaObserverChore extends ScheduledChore {
       final Multimap<String,TableName> tablesByNamespace) throws IOException {
     final SpaceQuotaStatus targetStatus = targetSnapshot.getQuotaStatus();
 
-    // When the policies differ, we need to move into or out of violatino
+    // When the policies differ, we need to move into or out of violation
     if (!currentSnapshot.equals(targetSnapshot)) {
       // We want to have a policy of "NONE", moving out of violation
       if (!targetStatus.isInViolation()) {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaUtil.java
index 99b6e1b..9053405 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaUtil.java
@@ -31,6 +31,9 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.TableNotDisabledException;
+import org.apache.hadoop.hbase.TableNotEnabledException;
+import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
@@ -500,4 +503,35 @@ public class QuotaUtil extends QuotaTableUtil {
     }
     return size;
   }
+
+  /**
+   * Method to enable a table, if not already enabled. This method suppresses
+   * {@link TableNotDisabledException} and {@link TableNotFoundException}, if thrown while enabling
+   * the table.
+   * @param conn connection to re-use
+   * @param tableName name of the table to be enabled
+   */
+  public static void enableTableIfNotEnabled(Connection conn, TableName tableName)
+      throws IOException {
+    try {
+      conn.getAdmin().enableTable(tableName);
+    } catch (TableNotDisabledException | TableNotFoundException e) {
+      // ignore
+    }
+  }
+
+  /**
+   * Method to disable a table, if not already disabled. This method suppresses
+   * {@link TableNotEnabledException}, if thrown while disabling the table.
+   * @param conn connection to re-use
+   * @param tableName table name which has moved into space quota violation
+   */
+  public static void disableTableIfNotDisabled(Connection conn, TableName tableName)
+      throws IOException {
+    try {
+      conn.getAdmin().disableTable(tableName);
+    } catch (TableNotEnabledException | TableNotFoundException e) {
+      // ignore
+    }
+  }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaRefresherChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaRefresherChore.java
index 045a44b..7ae7240 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaRefresherChore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/SpaceQuotaRefresherChore.java
@@ -96,18 +96,26 @@ public class SpaceQuotaRefresherChore extends ScheduledChore {
           LOG.trace(tableName + ": current=" + currentSnapshot + ", new=" + newSnapshot);
         }
         if (!newSnapshot.equals(currentSnapshot)) {
-          // We have a new snapshot. We might need to enforce it or disable the enforcement
-          if (!isInViolation(currentSnapshot) && newSnapshot.getQuotaStatus().isInViolation()) {
+          // We have a new snapshot.
+          // We might need to enforce it or disable the enforcement or switch policy
+          boolean currInViolation = isInViolation(currentSnapshot);
+          boolean newInViolation = newSnapshot.getQuotaStatus().isInViolation();
+          if (!currInViolation && newInViolation) {
             if (LOG.isTraceEnabled()) {
               LOG.trace("Enabling " + newSnapshot + " on " + tableName);
             }
             getManager().enforceViolationPolicy(tableName, newSnapshot);
-          }
-          if (isInViolation(currentSnapshot) && !newSnapshot.getQuotaStatus().isInViolation()) {
+          } else if (currInViolation && !newInViolation) {
             if (LOG.isTraceEnabled()) {
               LOG.trace("Removing quota violation policy on " + tableName);
             }
             getManager().disableViolationPolicyEnforcement(tableName);
+          } else if (currInViolation && newInViolation) {
+            if (LOG.isTraceEnabled()) {
+              LOG.trace("Switching quota violation policy on " + tableName + " from "
+                  + currentSnapshot + " to " + newSnapshot);
+            }
+            getManager().enforceViolationPolicy(tableName, newSnapshot);
           }
         }
       }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/DisableTableViolationPolicyEnforcement.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/DisableTableViolationPolicyEnforcement.java
index 9d24c92..b325f66 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/DisableTableViolationPolicyEnforcement.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/policies/DisableTableViolationPolicyEnforcement.java
@@ -18,55 +18,29 @@ package org.apache.hadoop.hbase.quotas.policies;
 
 import java.io.IOException;
 
-import org.apache.hadoop.hbase.TableNotDisabledException;
-import org.apache.hadoop.hbase.TableNotEnabledException;
-import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.quotas.SpaceLimitingException;
 import org.apache.hadoop.hbase.quotas.SpaceViolationPolicy;
 import org.apache.hadoop.hbase.quotas.SpaceViolationPolicyEnforcement;
 
 /**
- * A {@link SpaceViolationPolicyEnforcement} which disables the table. The enforcement
- * counterpart to {@link SpaceViolationPolicy#DISABLE}.
+ * A {@link SpaceViolationPolicyEnforcement} which disables the table. The enforcement counterpart
+ * to {@link SpaceViolationPolicy#DISABLE}. This violation policy is different from others as it
+ * doesn't take action (i.e. enable/disable table) local to the RegionServer, like the other
+ * ViolationPolicies do. In case of violation, the appropriate action is initiated by the master.
  */
 @InterfaceAudience.Private
 public class DisableTableViolationPolicyEnforcement extends DefaultViolationPolicyEnforcement {
-  private static final Logger LOG =
-      LoggerFactory.getLogger(DisableTableViolationPolicyEnforcement.class);
 
   @Override
   public void enable() throws IOException {
-    try {
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("Starting disable of " + getTableName());
-      }
-      getRegionServerServices().getClusterConnection().getAdmin().disableTable(getTableName());
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("Disable is complete for " + getTableName());
-      }
-    } catch (TableNotEnabledException tnee) {
-      // The state we wanted it to be in.
-    }
+    // do nothing
   }
 
   @Override
   public void disable() throws IOException {
-    try {
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("Starting enable of " + getTableName());
-      }
-      getRegionServerServices().getClusterConnection().getAdmin().enableTable(getTableName());
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("Enable is complete for " + getTableName());
-      }
-    } catch (TableNotDisabledException | TableNotFoundException e) {
-      // The state we wanted it to be in
-      // Or, in case table is not found, nothing to do
-    }
+    // do nothing
   }
 
   @Override
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotas.java
index 4b96f3d..05ee68a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotas.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestSpaceQuotas.java
@@ -381,7 +381,7 @@ public class TestSpaceQuotas {
   }
 
   @Test
-  public void testSetQuotaAndThenDropTableeWithNoWritesCompactions() throws Exception {
+  public void testSetQuotaAndThenDropTableWithNoWritesCompactions() throws Exception {
     setQuotaAndThenDropTable(SpaceViolationPolicy.NO_WRITES_COMPACTIONS);
   }
 
@@ -406,6 +406,16 @@ public class TestSpaceQuotas {
   }
 
   @Test
+  public void testSetQuotaAndThenIncreaseQuotaWithDisable() throws Exception {
+    setQuotaAndThenIncreaseQuota(SpaceViolationPolicy.DISABLE);
+  }
+
+  @Test
+  public void testSetQuotaAndThenDisableIncrEnableWithDisable() throws Exception {
+    setQuotaNextDisableThenIncreaseFinallyEnable(SpaceViolationPolicy.DISABLE);
+  }
+
+  @Test
   public void testSetQuotaAndThenRemoveInOneWithNoInserts() throws Exception {
     setQuotaAndThenRemoveInOneAmongTwoTables(SpaceViolationPolicy.NO_INSERTS);
   }
@@ -426,6 +436,36 @@ public class TestSpaceQuotas {
   }
 
   @Test
+  public void testSetQuotaFirstWithDisableNextNoWrites() throws Exception {
+    setQuotaAndViolateNextSwitchPoliciesAndValidate(SpaceViolationPolicy.DISABLE,
+      SpaceViolationPolicy.NO_WRITES);
+  }
+
+  @Test
+  public void testSetQuotaFirstWithDisableNextAgainDisable() throws Exception {
+    setQuotaAndViolateNextSwitchPoliciesAndValidate(SpaceViolationPolicy.DISABLE,
+      SpaceViolationPolicy.DISABLE);
+  }
+
+  @Test
+  public void testSetQuotaFirstWithDisableNextNoInserts() throws Exception {
+    setQuotaAndViolateNextSwitchPoliciesAndValidate(SpaceViolationPolicy.DISABLE,
+      SpaceViolationPolicy.NO_INSERTS);
+  }
+
+  @Test
+  public void testSetQuotaFirstWithDisableNextNoWritesCompaction() throws Exception {
+    setQuotaAndViolateNextSwitchPoliciesAndValidate(SpaceViolationPolicy.DISABLE,
+      SpaceViolationPolicy.NO_WRITES_COMPACTIONS);
+  }
+
+  @Test
+  public void testSetQuotaFirstWithNoWritesNextWithDisable() throws Exception {
+    setQuotaAndViolateNextSwitchPoliciesAndValidate(SpaceViolationPolicy.NO_WRITES,
+      SpaceViolationPolicy.DISABLE);
+  }
+
+  @Test
   public void testSetQuotaOnNonExistingTableWithNoInserts() throws Exception {
     setQuotaLimit(NON_EXISTENT_TABLE, SpaceViolationPolicy.NO_INSERTS, 2L);
   }
@@ -445,6 +485,26 @@ public class TestSpaceQuotas {
     setQuotaLimit(NON_EXISTENT_TABLE, SpaceViolationPolicy.DISABLE, 2L);
   }
 
+  public void setQuotaAndViolateNextSwitchPoliciesAndValidate(SpaceViolationPolicy policy1,
+      SpaceViolationPolicy policy2) throws Exception {
+    Put put = new Put(Bytes.toBytes("to_reject"));
+    put.addColumn(Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("to"),
+      Bytes.toBytes("reject"));
+
+    // Do puts until we violate space violation policy1
+    final TableName tn = writeUntilViolationAndVerifyViolation(policy1, put);
+
+    // Now, change violation policy to policy2
+    setQuotaLimit(tn, policy2, 2L);
+
+    // The table should be in enabled state on changing violation policy
+    if (policy1.equals(SpaceViolationPolicy.DISABLE) && !policy1.equals(policy2)) {
+      TEST_UTIL.waitTableEnabled(tn, 20000);
+    }
+    // Put some row now: should still violate as quota limit still violated
+    verifyViolation(policy2, tn, put);
+  }
+
   private void setQuotaAndThenRemove(SpaceViolationPolicy policy) throws Exception {
     Put put = new Put(Bytes.toBytes("to_reject"));
     put.addColumn(Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("to"),
@@ -495,6 +555,34 @@ public class TestSpaceQuotas {
     verifyNoViolation(policy, tn, put);
   }
 
+  private void setQuotaNextDisableThenIncreaseFinallyEnable(SpaceViolationPolicy policy)
+      throws Exception {
+    Put put = new Put(Bytes.toBytes("to_reject"));
+    put.addColumn(Bytes.toBytes(SpaceQuotaHelperForTests.F1), Bytes.toBytes("to"),
+      Bytes.toBytes("reject"));
+
+    // Do puts until we violate space policy
+    final TableName tn = writeUntilViolationAndVerifyViolation(policy, put);
+
+    // Disable the table; in case of SpaceViolationPolicy.DISABLE already disabled
+    if (!policy.equals(SpaceViolationPolicy.DISABLE)) {
+      TEST_UTIL.getAdmin().disableTable(tn);
+      TEST_UTIL.waitTableDisabled(tn, 10000);
+    }
+
+    // Now, increase limit and perform put
+    setQuotaLimit(tn, policy, 4L);
+
+    // in case of disable policy quota manager will enable it
+    if (!policy.equals(SpaceViolationPolicy.DISABLE)) {
+      TEST_UTIL.getAdmin().enableTable(tn);
+    }
+    TEST_UTIL.waitTableEnabled(tn, 10000);
+
+    // Put some row now: should not violate as quota limit increased
+    verifyNoViolation(policy, tn, put);
+  }
+
   public void setQuotaAndThenRemoveInOneAmongTwoTables(SpaceViolationPolicy policy)
       throws Exception {
     Put put = new Put(Bytes.toBytes("to_reject"));
@@ -565,6 +653,7 @@ public class TestSpaceQuotas {
 			SpaceViolationPolicy policyToViolate, TableName tn, Mutation m) throws Exception {
     // But let's try a few times to get the exception before failing
     boolean sawError = false;
+    String msg = "";
     for (int i = 0; i < NUM_RETRIES && !sawError; i++) {
       try (Table table = TEST_UTIL.getConnection().getTable(tn)) {
         if (m instanceof Put) {
@@ -583,15 +672,16 @@ public class TestSpaceQuotas {
         LOG.info("Did not reject the " + m.getClass().getSimpleName() + ", will sleep and retry");
         Thread.sleep(2000);
       } catch (Exception e) {
-        String msg = StringUtils.stringifyException(e);
-        if (policyToViolate.equals(SpaceViolationPolicy.DISABLE)) {
-          assertTrue(e instanceof TableNotEnabledException);
+        msg = StringUtils.stringifyException(e);
+        if ((policyToViolate.equals(SpaceViolationPolicy.DISABLE)
+            && e instanceof TableNotEnabledException) || msg.contains(policyToViolate.name())) {
+          LOG.info("Got the expected exception={}", msg);
+          sawError = true;
+          break;
         } else {
-          assertTrue("Expected exception message to contain the word '" + policyToViolate.name()
-              + "', but was " + msg,
-            msg.contains(policyToViolate.name()));
+          LOG.warn("Did not get the expected exception, will sleep and retry", e);
+          Thread.sleep(2000);
         }
-        sawError = true;
       }
     }
     if (!sawError) {
@@ -604,6 +694,15 @@ public class TestSpaceQuotas {
         }
         scanner.close();
       }
+    } else {
+      if (policyToViolate.equals(SpaceViolationPolicy.DISABLE)) {
+        assertTrue(
+          msg.contains("TableNotEnabledException") || msg.contains(policyToViolate.name()));
+      } else {
+        assertTrue("Expected exception message to contain the word '" + policyToViolate.name()
+            + "', but was " + msg,
+          msg.contains(policyToViolate.name()));
+      }
     }
     assertTrue(
         "Expected to see an exception writing data to a table exceeding its quota", sawError);


[hbase] 08/49: HBASE-22032 KeyValue validation should check for null byte array

Posted by op...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

openinx pushed a commit to branch HBASE-21879
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit e5dd8d2a1d5f03587ae587c78cdcbf968f55e839
Author: Geoffrey Jacoby <gj...@apache.org>
AuthorDate: Mon Mar 11 14:08:49 2019 -0700

    HBASE-22032 KeyValue validation should check for null byte array
    
    Signed-off-by: Andrew Purtell <ap...@apache.org>
---
 .../main/java/org/apache/hadoop/hbase/KeyValueUtil.java  |  8 +++++++-
 .../test/java/org/apache/hadoop/hbase/TestKeyValue.java  | 16 +++++++++++++++-
 2 files changed, 22 insertions(+), 2 deletions(-)

diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java
index 581c6bb..0a6d63a 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueUtil.java
@@ -509,11 +509,17 @@ public class KeyValueUtil {
   }
 
   static String bytesToHex(byte[] buf, int offset, int length) {
-    return ", KeyValueBytesHex=" + Bytes.toStringBinary(buf, offset, length) + ", offset=" + offset
+    String bufferContents = buf != null ? Bytes.toStringBinary(buf, offset, length) : "<null>";
+    return ", KeyValueBytesHex=" + bufferContents + ", offset=" + offset
         + ", length=" + length;
   }
 
   static void checkKeyValueBytes(byte[] buf, int offset, int length, boolean withTags) {
+    if (buf == null) {
+      throw new IllegalArgumentException("Invalid to have null " +
+          "byte array in KeyValue.");
+    }
+
     int pos = offset, endOffset = offset + length;
     // check the key
     if (pos + Bytes.SIZEOF_INT > endOffset) {
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java
index 2165b38..e14fdf2 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java
@@ -623,6 +623,20 @@ public class TestKeyValue {
     }
   }
 
+  @Test
+  public void testNullByteArrayKeyValueFailure() {
+    //can't add to testCheckKeyValueBytesFailureCase because it
+    //goes through the InputStream KeyValue API which can't produce a null buffer
+    try {
+      KeyValue kv = new KeyValue(null, 0, 0);
+    } catch (IllegalArgumentException iae){
+      assertEquals("Invalid to have null byte array in KeyValue.", iae.getMessage());
+      return;
+    }
+    fail("Should have thrown an IllegalArgumentException when " +
+        "creating a KeyValue with a null buffer");
+  }
+
   private static class FailureCase {
     byte[] buf;
     int offset;
@@ -681,7 +695,7 @@ public class TestKeyValue {
       "Overflow when reading family length at position=13",
       "Invalid family length in KeyValue, familyLength=1", "Timestamp cannot be negative, ts=-1",
       "Invalid type in KeyValue, type=3", "Overflow when reading value part at position=25",
-      "Invalid tags length in KeyValue at position=26", };
+      "Invalid tags length in KeyValue at position=26"};
     byte[][] withTagsInputs = new byte[][] {
       Bytes.toBytesBinary("\\x00\\x00\\x00\\x11\\x00\\x00\\x00\\x01\\x00\\x03ROW\\x01FQ\\x00"
           + "\\x00\\x00\\x00\\x00\\x00\\x00\\x01\\x04V\\x01"), // case.13


[hbase] 09/49: HBASE-22039 Should add the synchronous parameter for the XXXSwitch method in AsyncAdmin

Posted by op...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

openinx pushed a commit to branch HBASE-21879
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit 44d4723bf6f7df8af8b2c2de62b21d1978e54206
Author: Duo Zhang <zh...@apache.org>
AuthorDate: Wed Mar 13 17:32:45 2019 +0800

    HBASE-22039 Should add the synchronous parameter for the XXXSwitch method in AsyncAdmin
    
    Signed-off-by: Zheng Hu <op...@gmail.com>
---
 .../org/apache/hadoop/hbase/client/AsyncAdmin.java | 57 +++++++++++++++++++---
 .../hadoop/hbase/client/AsyncHBaseAdmin.java       | 12 ++---
 .../hadoop/hbase/client/RawAsyncHBaseAdmin.java    | 43 ++++++++--------
 3 files changed, 78 insertions(+), 34 deletions(-)

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 5952821..623e4f8 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
@@ -435,10 +435,25 @@ public interface AsyncAdmin {
 
   /**
    * Turn the Merge switch on or off.
-   * @param on
+   * @param enabled enabled or not
+   * @return Previous switch value wrapped by a {@link CompletableFuture}
+   */
+  default CompletableFuture<Boolean> mergeSwitch(boolean enabled) {
+    return mergeSwitch(enabled, false);
+  }
+
+  /**
+   * Turn the Merge switch on or off.
+   * <p/>
+   * Notice that, the method itself is always non-blocking, which means it will always return
+   * immediately. The {@code drainMerges} parameter only effects when will we complete the returned
+   * {@link CompletableFuture}.
+   * @param enabled enabled or not
+   * @param drainMerges If <code>true</code>, it waits until current merge() call, if outstanding,
+   *          to return.
    * @return Previous switch value wrapped by a {@link CompletableFuture}
    */
-  CompletableFuture<Boolean> mergeSwitch(boolean on);
+  CompletableFuture<Boolean> mergeSwitch(boolean enabled, boolean drainMerges);
 
   /**
    * Query the current state of the Merge switch.
@@ -449,10 +464,25 @@ public interface AsyncAdmin {
 
   /**
    * Turn the Split switch on or off.
-   * @param on
+   * @param enabled enabled or not
+   * @return Previous switch value wrapped by a {@link CompletableFuture}
+   */
+  default CompletableFuture<Boolean> splitSwitch(boolean enabled) {
+    return splitSwitch(enabled, false);
+  }
+
+  /**
+   * Turn the Split switch on or off.
+   * <p/>
+   * Notice that, the method itself is always non-blocking, which means it will always return
+   * immediately. The {@code drainSplits} parameter only effects when will we complete the returned
+   * {@link CompletableFuture}.
+   * @param enabled enabled or not
+   * @param drainSplits If <code>true</code>, it waits until current split() call, if outstanding,
+   *          to return.
    * @return Previous switch value wrapped by a {@link CompletableFuture}
    */
-  CompletableFuture<Boolean> splitSwitch(boolean on);
+  CompletableFuture<Boolean> splitSwitch(boolean enabled, boolean drainSplits);
 
   /**
    * Query the current state of the Split switch.
@@ -1132,10 +1162,25 @@ public interface AsyncAdmin {
 
   /**
    * Turn the load balancer on or off.
-   * @param on
+   * @param on Set to <code>true</code> to enable, <code>false</code> to disable.
+   * @return Previous balancer value wrapped by a {@link CompletableFuture}.
+   */
+  default CompletableFuture<Boolean> balancerSwitch(boolean on) {
+    return balancerSwitch(on, false);
+  }
+
+  /**
+   * Turn the load balancer on or off.
+   * <p/>
+   * Notice that, the method itself is always non-blocking, which means it will always return
+   * immediately. The {@code drainRITs} parameter only effects when will we complete the returned
+   * {@link CompletableFuture}.
+   * @param on Set to <code>true</code> to enable, <code>false</code> to disable.
+   * @param drainRITs If <code>true</code>, it waits until current balance() call, if outstanding,
+   *          to return.
    * @return Previous balancer value wrapped by a {@link CompletableFuture}.
    */
-  CompletableFuture<Boolean> balancerSwitch(boolean on);
+  CompletableFuture<Boolean> balancerSwitch(boolean on, boolean drainRITs);
 
   /**
    * Invoke the balancer. Will run the balancer and if regions to move, it will go ahead and do the
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 53eaec8..ab37b31 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
@@ -296,8 +296,8 @@ class AsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<Boolean> mergeSwitch(boolean on) {
-    return wrap(rawAdmin.mergeSwitch(on));
+  public CompletableFuture<Boolean> mergeSwitch(boolean enabled, boolean drainMerges) {
+    return wrap(rawAdmin.mergeSwitch(enabled, drainMerges));
   }
 
   @Override
@@ -306,8 +306,8 @@ class AsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<Boolean> splitSwitch(boolean on) {
-    return wrap(rawAdmin.splitSwitch(on));
+  public CompletableFuture<Boolean> splitSwitch(boolean enabled, boolean drainSplits) {
+    return wrap(rawAdmin.splitSwitch(enabled, drainSplits));
   }
 
   @Override
@@ -664,8 +664,8 @@ class AsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<Boolean> balancerSwitch(boolean on) {
-    return wrap(rawAdmin.balancerSwitch(on));
+  public CompletableFuture<Boolean> balancerSwitch(boolean on, boolean drainRITs) {
+    return wrap(rawAdmin.balancerSwitch(on, drainRITs));
   }
 
   @Override
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 1092332..c73eec4 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
@@ -1176,8 +1176,8 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<Boolean> mergeSwitch(boolean on) {
-    return setSplitOrMergeOn(on, MasterSwitchType.MERGE);
+  public CompletableFuture<Boolean> mergeSwitch(boolean enabled, boolean drainMerges) {
+    return setSplitOrMergeOn(enabled, drainMerges, MasterSwitchType.MERGE);
   }
 
   @Override
@@ -1186,8 +1186,8 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<Boolean> splitSwitch(boolean on) {
-    return setSplitOrMergeOn(on, MasterSwitchType.SPLIT);
+  public CompletableFuture<Boolean> splitSwitch(boolean enabled, boolean drainSplits) {
+    return setSplitOrMergeOn(enabled, drainSplits, MasterSwitchType.SPLIT);
   }
 
   @Override
@@ -1195,16 +1195,16 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
     return isSplitOrMergeOn(MasterSwitchType.SPLIT);
   }
 
-  private CompletableFuture<Boolean> setSplitOrMergeOn(boolean on, MasterSwitchType switchType) {
+  private CompletableFuture<Boolean> setSplitOrMergeOn(boolean enabled, boolean synchronous,
+      MasterSwitchType switchType) {
     SetSplitOrMergeEnabledRequest request =
-        RequestConverter.buildSetSplitOrMergeEnabledRequest(on, false, switchType);
-    return this
-        .<Boolean> newMasterCaller()
-        .action(
-          (controller, stub) -> this
-              .<SetSplitOrMergeEnabledRequest, SetSplitOrMergeEnabledResponse, Boolean> call(
-                controller, stub, request, (s, c, req, done) -> s.setSplitOrMergeEnabled(c, req,
-                  done), (resp) -> resp.getPrevValueList().get(0))).call();
+      RequestConverter.buildSetSplitOrMergeEnabledRequest(enabled, synchronous, switchType);
+    return this.<Boolean> newMasterCaller()
+      .action((controller, stub) -> this
+        .<SetSplitOrMergeEnabledRequest, SetSplitOrMergeEnabledResponse, Boolean> call(controller,
+          stub, request, (s, c, req, done) -> s.setSplitOrMergeEnabled(c, req, done),
+          (resp) -> resp.getPrevValueList().get(0)))
+      .call();
   }
 
   private CompletableFuture<Boolean> isSplitOrMergeOn(MasterSwitchType switchType) {
@@ -3127,15 +3127,14 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<Boolean> balancerSwitch(final boolean on) {
-    return this
-        .<Boolean> newMasterCaller()
-        .action(
-          (controller, stub) -> this
-              .<SetBalancerRunningRequest, SetBalancerRunningResponse, Boolean> call(controller,
-                stub, RequestConverter.buildSetBalancerRunningRequest(on, true),
-                (s, c, req, done) -> s.setBalancerRunning(c, req, done),
-                (resp) -> resp.getPrevBalanceValue())).call();
+  public CompletableFuture<Boolean> balancerSwitch(boolean on, boolean drainRITs) {
+    return this.<Boolean> newMasterCaller()
+      .action((controller, stub) -> this
+        .<SetBalancerRunningRequest, SetBalancerRunningResponse, Boolean> call(controller, stub,
+          RequestConverter.buildSetBalancerRunningRequest(on, drainRITs),
+          (s, c, req, done) -> s.setBalancerRunning(c, req, done),
+          (resp) -> resp.getPrevBalanceValue()))
+      .call();
   }
 
   @Override


[hbase] 35/49: Add 2.1.4 to the downloads page in place of 2.1.3; ADDENDUM

Posted by op...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

openinx pushed a commit to branch HBASE-21879
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit 6ec0c6b292311b3cea789c6984793d8260242146
Author: stack <st...@apache.org>
AuthorDate: Tue Mar 26 10:59:04 2019 -0400

       Add 2.1.4 to the downloads page in place of 2.1.3; ADDENDUM
---
 src/site/xdoc/downloads.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/src/site/xdoc/downloads.xml b/src/site/xdoc/downloads.xml
index c53e9f1..365fe9c 100644
--- a/src/site/xdoc/downloads.xml
+++ b/src/site/xdoc/downloads.xml
@@ -60,7 +60,7 @@ under the License.
       </td>
       <td style="test-align: left">
         <a href="https://www.apache.org/dyn/closer.lua/hbase/2.1.4/hbase-2.1.4-src.tar.gz">src</a> (<a href="https://apache.org/dist/hbase/2.1.4/hbase-2.1.4-src.tar.gz.sha512">sha512</a> <a href="https://apache.org/dist/hbase/2.1.4/hbase-2.1.4-src.tar.gz.asc">asc</a>) <br />
-        <a href=4https://www.apache.org/dyn/closer.lua/hbase/2.1.4/hbase-2.1.4-bin.tar.gz">bin</a> (<a href="https://apache.org/dist/hbase/2.1.4/hbase-2.1.4-bin.tar.gz.sha512">sha512</a> <a href="https://apache.org/dist/hbase/2.1.4/hbase-2.1.4-bin.tar.gz.asc">asc</a>) <br />
+        <a href="https://www.apache.org/dyn/closer.lua/hbase/2.1.4/hbase-2.1.4-bin.tar.gz">bin</a> (<a href="https://apache.org/dist/hbase/2.1.4/hbase-2.1.4-bin.tar.gz.sha512">sha512</a> <a href="https://apache.org/dist/hbase/2.1.4/hbase-2.1.4-bin.tar.gz.asc">asc</a>) <br />
         <a href="https://www.apache.org/dyn/closer.lua/hbase/2.1.4/hbase-2.1.4-client-bin.tar.gz">client-bin</a> (<a href="https://apache.org/dist/hbase/2.1.4/hbase-2.1.4-client-bin.tar.gz.sha512">sha512</a> <a href="https://apache.org/dist/hbase/2.1.4/hbase-2.1.4-client-bin.tar.gz.asc">asc</a>)
       </td>
     </tr>


[hbase] 38/49: Add a 'notices' column to downloads where we can call out issues

Posted by op...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

openinx pushed a commit to branch HBASE-21879
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit a3edb1814d0fc86eaf7fce11b1e8829564114bc3
Author: stack <st...@apache.org>
AuthorDate: Tue Mar 26 18:25:21 2019 -0700

    Add a 'notices' column to downloads where we can call out issues
---
 src/site/xdoc/downloads.xml | 7 +++++++
 1 file changed, 7 insertions(+)

diff --git a/src/site/xdoc/downloads.xml b/src/site/xdoc/downloads.xml
index 38f17e2..e2e58f1 100644
--- a/src/site/xdoc/downloads.xml
+++ b/src/site/xdoc/downloads.xml
@@ -41,6 +41,7 @@ under the License.
       <th>Changes</th>
       <th>Release Notes</th>
       <th>Download</th>
+      <th>Notices</th>
     </tr>
     <tr>
       <td style="test-align: left">
@@ -63,6 +64,8 @@ under the License.
         <a href="https://www.apache.org/dyn/closer.lua/hbase/2.1.4/hbase-2.1.4-bin.tar.gz">bin</a> (<a href="https://apache.org/dist/hbase/2.1.4/hbase-2.1.4-bin.tar.gz.sha512">sha512</a> <a href="https://apache.org/dist/hbase/2.1.4/hbase-2.1.4-bin.tar.gz.asc">asc</a>) <br />
         <a href="https://www.apache.org/dyn/closer.lua/hbase/2.1.4/hbase-2.1.4-client-bin.tar.gz">client-bin</a> (<a href="https://apache.org/dist/hbase/2.1.4/hbase-2.1.4-client-bin.tar.gz.sha512">sha512</a> <a href="https://apache.org/dist/hbase/2.1.4/hbase-2.1.4-client-bin.tar.gz.asc">asc</a>)
       </td>
+      <td>The shaded hbase-client might be broke in 2.1.4; you may need to specify an old Apache HTrace dependency. See HBASE-22052
+      </td>
     </tr>
     <tr>
       <td style="test-align: left">
@@ -84,6 +87,7 @@ under the License.
         <a href="https://www.apache.org/dyn/closer.lua/hbase/2.0.5/hbase-2.0.5-src.tar.gz">src</a> (<a href="https://dist.apache.org/repos/dist/release/hbase/2.0.5/hbase-2.0.5-src.tar.gz.sha512">sha512</a> <a href="https://dist.apache.org/repos/dist/release/hbase/2.0.5/hbase-2.0.5-src.tar.gz.asc">asc</a>) <br />
         <a href="https://www.apache.org/dyn/closer.lua/hbase/2.0.5/hbase-2.0.5-bin.tar.gz">bin</a> (<a href="https://dist.apache.org/repos/dist/release/hbase/2.0.5/hbase-2.0.5-bin.tar.gz.sha512">sha512</a> <a href="https://dist.apache.org/repos/dist/release/hbase/2.0.5/hbase-2.0.5-bin.tar.gz.asc">asc</a>)
       </td>
+      <td />
     </tr>
     <tr>
       <td style="test-align: left">
@@ -105,6 +109,7 @@ under the License.
         <a href="https://www.apache.org/dyn/closer.lua/hbase/1.4.9/hbase-1.4.9-src.tar.gz">src</a> (<a href="https://apache.org/dist/hbase/1.4.9/hbase-1.4.9-src.tar.gz.sha512">sha512</a> <a href="https://apache.org/dist/hbase/1.4.9/hbase-1.4.9-src.tar.gz.asc">asc</a>) <br />
         <a href="https://www.apache.org/dyn/closer.lua/hbase/1.4.9/hbase-1.4.9-bin.tar.gz">bin</a> (<a href="https://apache.org/dist/hbase/1.4.9/hbase-1.4.9-bin.tar.gz.sha512">sha512</a> <a href="https://apache.org/dist/hbase/1.4.9/hbase-1.4.9-bin.tar.gz.asc">asc</a>)
       </td>
+      <td />
     </tr>
     <tr>
       <td style="test-align: left">
@@ -126,6 +131,7 @@ under the License.
         <a href="https://www.apache.org/dyn/closer.lua/hbase/1.3.3/hbase-1.3.3-src.tar.gz">src</a> (<a href="https://apache.org/dist/hbase/1.3.3/hbase-1.3.3-src.tar.gz.sha512">sha512</a> <a href="https://apache.org/dist/hbase/1.3.3/hbase-1.3.3-src.tar.gz.asc">asc</a>) <br />
         <a href="https://www.apache.org/dyn/closer.lua/hbase/1.3.3/hbase-1.3.3-bin.tar.gz">bin</a> (<a href="https://apache.org/dist/hbase/1.3.3/hbase-1.3.3-bin.tar.gz.sha512">sha512</a> <a href="https://apache.org/dist/hbase/1.3.3/hbase-1.3.3-bin.tar.gz.asc">asc</a>)
       </td>
+      <td />
     </tr>
     <tr>
       <td style="test-align: left">
@@ -147,6 +153,7 @@ under the License.
         <a href="https://www.apache.org/dyn/closer.lua/hbase/hbase-1.2.11/hbase-1.2.11-src.tar.gz">src</a> (<a href="https://apache.org/dist/hbase/hbase-1.2.11/hbase-1.2.11-src.tar.gz.sha512">sha512</a> <a href="https://apache.org/dist/hbase/hbase-1.2.11/hbase-1.2.11-src.tar.gz.asc">asc</a>) <br />
         <a href="https://www.apache.org/dyn/closer.lua/hbase/hbase-1.2.11/hbase-1.2.11-bin.tar.gz">bin</a> (<a href="https://apache.org/dist/hbase/hbase-1.2.11/hbase-1.2.11-bin.tar.gz.sha512">sha512</a> <a href="https://apache.org/dist/hbase/hbase-1.2.11/hbase-1.2.11-bin.tar.gz.asc">asc</a>)
       </td>
+      <td />
     </tr>
   </table>
   </section>


[hbase] 11/49: HBASE-22040 Add mergeRegionsAsync with a List of region names method in AsyncAdmin

Posted by op...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

openinx pushed a commit to branch HBASE-21879
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit a67d08bb849f97cf1b7bdcb1dbd24ef05c9c517c
Author: zhangduo <zh...@apache.org>
AuthorDate: Sat Mar 16 10:25:34 2019 +0800

    HBASE-22040 Add mergeRegionsAsync with a List of region names method in AsyncAdmin
    
    Signed-off-by: Zheng Hu <op...@gmail.com>
---
 .../java/org/apache/hadoop/hbase/client/Admin.java | 28 +++++----
 .../org/apache/hadoop/hbase/client/AsyncAdmin.java | 19 +++++-
 .../hadoop/hbase/client/AsyncHBaseAdmin.java       |  5 +-
 .../org/apache/hadoop/hbase/client/HBaseAdmin.java | 37 +++--------
 .../hadoop/hbase/client/RawAsyncHBaseAdmin.java    | 72 +++++++++++-----------
 .../hadoop/hbase/master/MasterRpcServices.java     |  5 +-
 .../org/apache/hadoop/hbase/client/TestAdmin1.java | 46 ++++++++++++++
 .../hbase/client/TestAsyncRegionAdminApi2.java     | 58 ++++++++++++++---
 8 files changed, 178 insertions(+), 92 deletions(-)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
index b65b1c4..e595e51 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
@@ -830,29 +830,31 @@ public interface Admin extends Abortable, Closeable {
 
   /**
    * Merge two regions. Asynchronous operation.
-   *
    * @param nameOfRegionA encoded or full name of region a
    * @param nameOfRegionB encoded or full name of region b
-   * @param forcible <code>true</code> if do a compulsory merge, otherwise we will only merge
-   *          two adjacent regions
-   * @throws IOException
+   * @param forcible <code>true</code> if do a compulsory merge, otherwise we will only merge two
+   *          adjacent regions
    */
-  Future<Void> mergeRegionsAsync(
-      byte[] nameOfRegionA,
-      byte[] nameOfRegionB,
-      boolean forcible) throws IOException;
+  default Future<Void> mergeRegionsAsync(byte[] nameOfRegionA, byte[] nameOfRegionB,
+      boolean forcible) throws IOException {
+    byte[][] nameofRegionsToMerge = new byte[2][];
+    nameofRegionsToMerge[0] = nameOfRegionA;
+    nameofRegionsToMerge[1] = nameOfRegionB;
+    return mergeRegionsAsync(nameofRegionsToMerge, forcible);
+  }
 
   /**
    * Merge regions. Asynchronous operation.
-   *
+   * <p/>
+   * You may get a {@code DoNotRetryIOException} if you pass more than two regions in but the master
+   * does not support merging more than two regions. At least till 2.2.0, we still only support
+   * merging two regions.
    * @param nameofRegionsToMerge encoded or full name of daughter regions
    * @param forcible <code>true</code> if do a compulsory merge, otherwise we will only merge
    *          adjacent regions
-   * @throws IOException
    */
-  Future<Void> mergeRegionsAsync(
-      byte[][] nameofRegionsToMerge,
-      boolean forcible) throws IOException;
+  Future<Void> mergeRegionsAsync(byte[][] nameofRegionsToMerge, boolean forcible)
+      throws IOException;
 
   /**
    * Split a table. The method will execute split action for each region in table.
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 623e4f8..a4994e6 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
@@ -21,6 +21,7 @@ import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
 
 import com.google.protobuf.RpcChannel;
 import java.io.IOException;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.EnumSet;
 import java.util.List;
@@ -498,8 +499,22 @@ public interface AsyncAdmin {
    * @param forcible true if do a compulsory merge, otherwise we will only merge two adjacent
    *          regions
    */
-  CompletableFuture<Void> mergeRegions(byte[] nameOfRegionA, byte[] nameOfRegionB,
-      boolean forcible);
+  default CompletableFuture<Void> mergeRegions(byte[] nameOfRegionA, byte[] nameOfRegionB,
+      boolean forcible) {
+    return mergeRegions(Arrays.asList(nameOfRegionA, nameOfRegionB), forcible);
+  }
+
+  /**
+   * Merge regions.
+   * <p/>
+   * You may get a {@code DoNotRetryIOException} if you pass more than two regions in but the master
+   * does not support merging more than two regions. At least till 2.2.0, we still only support
+   * merging two regions.
+   * @param nameOfRegionsToMerge encoded or full name of daughter regions
+   * @param forcible true if do a compulsory merge, otherwise we will only merge two adjacent
+   *          regions
+   */
+  CompletableFuture<Void> mergeRegions(List<byte[]> nameOfRegionsToMerge, boolean forcible);
 
   /**
    * Split a table. The method will execute split action for each region in table.
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 ab37b31..41095d0 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
@@ -316,9 +316,8 @@ class AsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<Void> mergeRegions(byte[] nameOfRegionA, byte[] nameOfRegionB,
-      boolean forcible) {
-    return wrap(rawAdmin.mergeRegions(nameOfRegionA, nameOfRegionB, forcible));
+  public CompletableFuture<Void> mergeRegions(List<byte[]> nameOfRegionsToMerge, boolean forcible) {
+    return wrap(rawAdmin.mergeRegions(nameOfRegionsToMerge, forcible));
   }
 
   @Override
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 3a0588b..02ba1d0 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
@@ -106,6 +106,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
 import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
 
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
@@ -1425,40 +1426,20 @@ public class HBaseAdmin implements Admin {
 
   /**
    * Merge two regions. Asynchronous operation.
-   * @param nameOfRegionA encoded or full name of region a
-   * @param nameOfRegionB encoded or full name of region b
-   * @param forcible true if do a compulsory merge, otherwise we will only merge
-   *          two adjacent regions
-   * @throws IOException
-   */
-  @Override
-  public Future<Void> mergeRegionsAsync(
-      final byte[] nameOfRegionA,
-      final byte[] nameOfRegionB,
-      final boolean forcible) throws IOException {
-    byte[][] nameofRegionsToMerge = new byte[2][];
-    nameofRegionsToMerge[0] = nameOfRegionA;
-    nameofRegionsToMerge[1] = nameOfRegionB;
-    return mergeRegionsAsync(nameofRegionsToMerge, forcible);
-  }
-
-  /**
-   * Merge two regions. Asynchronous operation.
    * @param nameofRegionsToMerge encoded or full name of daughter regions
    * @param forcible true if do a compulsory merge, otherwise we will only merge
    *          adjacent regions
-   * @throws IOException
    */
   @Override
-  public Future<Void> mergeRegionsAsync(
-      final byte[][] nameofRegionsToMerge,
-      final boolean forcible) throws IOException {
-    assert(nameofRegionsToMerge.length >= 2);
+  public Future<Void> mergeRegionsAsync(final byte[][] nameofRegionsToMerge, final boolean forcible)
+      throws IOException {
+    Preconditions.checkArgument(nameofRegionsToMerge.length >= 2, "Can not merge only %s region",
+      nameofRegionsToMerge.length);
     byte[][] encodedNameofRegionsToMerge = new byte[nameofRegionsToMerge.length][];
-    for(int i = 0; i < nameofRegionsToMerge.length; i++) {
-      encodedNameofRegionsToMerge[i] = RegionInfo.isEncodedRegionName(nameofRegionsToMerge[i]) ?
-          nameofRegionsToMerge[i] :
-          Bytes.toBytes(RegionInfo.encodeRegionName(nameofRegionsToMerge[i]));
+    for (int i = 0; i < nameofRegionsToMerge.length; i++) {
+      encodedNameofRegionsToMerge[i] =
+        RegionInfo.isEncodedRegionName(nameofRegionsToMerge[i]) ? nameofRegionsToMerge[i]
+          : Bytes.toBytes(RegionInfo.encodeRegionName(nameofRegionsToMerge[i]));
     }
 
     TableName tableName = null;
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 c73eec4..7a97ce8 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
@@ -1165,13 +1165,12 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
     });
   }
 
-  private CompletableFuture<TableName> checkRegionsAndGetTableName(byte[] encodeRegionNameA,
-      byte[] encodeRegionNameB) {
+  private CompletableFuture<TableName> checkRegionsAndGetTableName(byte[][] encodedRegionNames) {
     AtomicReference<TableName> tableNameRef = new AtomicReference<>();
     CompletableFuture<TableName> future = new CompletableFuture<>();
-
-    checkAndGetTableName(encodeRegionNameA, tableNameRef, future);
-    checkAndGetTableName(encodeRegionNameB, tableNameRef, future);
+    for (byte[] encodedRegionName : encodedRegionNames) {
+      checkAndGetTableName(encodedRegionName, tableNameRef, future);
+    }
     return future;
   }
 
@@ -1221,41 +1220,42 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
-  public CompletableFuture<Void> mergeRegions(byte[] nameOfRegionA, byte[] nameOfRegionB,
-      boolean forcible) {
+  public CompletableFuture<Void> mergeRegions(List<byte[]> nameOfRegionsToMerge, boolean forcible) {
+    if (nameOfRegionsToMerge.size() < 2) {
+      return failedFuture(new IllegalArgumentException(
+        "Can not merge only " + nameOfRegionsToMerge.size() + " region"));
+    }
     CompletableFuture<Void> future = new CompletableFuture<>();
-    final byte[] encodeRegionNameA = toEncodeRegionName(nameOfRegionA);
-    final byte[] encodeRegionNameB = toEncodeRegionName(nameOfRegionB);
+    byte[][] encodedNameOfRegionsToMerge =
+      nameOfRegionsToMerge.stream().map(this::toEncodeRegionName).toArray(byte[][]::new);
 
-    addListener(checkRegionsAndGetTableName(encodeRegionNameA, encodeRegionNameB),
-      (tableName, err) -> {
-        if (err != null) {
-          future.completeExceptionally(err);
-          return;
-        }
+    addListener(checkRegionsAndGetTableName(encodedNameOfRegionsToMerge), (tableName, err) -> {
+      if (err != null) {
+        future.completeExceptionally(err);
+        return;
+      }
 
-        MergeTableRegionsRequest request = null;
-        try {
-          request = RequestConverter.buildMergeTableRegionsRequest(
-            new byte[][] { encodeRegionNameA, encodeRegionNameB }, forcible, ng.getNonceGroup(),
-            ng.newNonce());
-        } catch (DeserializationException e) {
-          future.completeExceptionally(e);
-          return;
-        }
+      MergeTableRegionsRequest request = null;
+      try {
+        request = RequestConverter.buildMergeTableRegionsRequest(encodedNameOfRegionsToMerge,
+          forcible, ng.getNonceGroup(), ng.newNonce());
+      } catch (DeserializationException e) {
+        future.completeExceptionally(e);
+        return;
+      }
 
-        addListener(
-          this.<MergeTableRegionsRequest, MergeTableRegionsResponse> procedureCall(tableName,
-            request, (s, c, req, done) -> s.mergeTableRegions(c, req, done),
-            (resp) -> resp.getProcId(), new MergeTableRegionProcedureBiConsumer(tableName)),
-          (ret, err2) -> {
-            if (err2 != null) {
-              future.completeExceptionally(err2);
-            } else {
-              future.complete(ret);
-            }
-          });
-      });
+      addListener(
+        this.<MergeTableRegionsRequest, MergeTableRegionsResponse> procedureCall(tableName, request,
+          (s, c, req, done) -> s.mergeTableRegions(c, req, done), (resp) -> resp.getProcId(),
+          new MergeTableRegionProcedureBiConsumer(tableName)),
+        (ret, err2) -> {
+          if (err2 != null) {
+            future.completeExceptionally(err2);
+          } else {
+            future.complete(ret);
+          }
+        });
+    });
     return future;
   }
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index 063a353..b943000 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -794,7 +794,10 @@ public class MasterRpcServices extends RSRpcServices
 
     RegionStates regionStates = master.getAssignmentManager().getRegionStates();
 
-    assert(request.getRegionCount() == 2);
+    if (request.getRegionCount() != 2) {
+      throw new ServiceException(new DoNotRetryIOException(
+        "Only support merging 2 regions but " + request.getRegionCount() + " region passed"));
+    }
     RegionInfo[] regionsToMerge = new RegionInfo[request.getRegionCount()];
     for (int i = 0; i < request.getRegionCount(); i++) {
       final byte[] encodedNameOfRegion = request.getRegion(i).getValue().toByteArray();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
index d86be09..87db47a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
@@ -29,9 +29,11 @@ import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.regex.Pattern;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
@@ -1419,6 +1421,50 @@ public class TestAdmin1 {
   }
 
   @Test
+  public void testMergeRegionsInvalidRegionCount()
+      throws IOException, InterruptedException, ExecutionException {
+    TableName tableName = TableName.valueOf(name.getMethodName());
+    TableDescriptor td = TableDescriptorBuilder.newBuilder(tableName)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of("d")).build();
+    byte[][] splitRows = new byte[2][];
+    splitRows[0] = new byte[] { (byte) '3' };
+    splitRows[1] = new byte[] { (byte) '6' };
+    try {
+      TEST_UTIL.createTable(td, splitRows);
+      TEST_UTIL.waitTableAvailable(tableName);
+
+      List<RegionInfo> tableRegions = ADMIN.getRegions(tableName);
+      // 0
+      try {
+        ADMIN.mergeRegionsAsync(new byte[0][0], false).get();
+        fail();
+      } catch (IllegalArgumentException e) {
+        // expected
+      }
+      // 1
+      try {
+        ADMIN.mergeRegionsAsync(new byte[][] { tableRegions.get(0).getEncodedNameAsBytes() }, false)
+          .get();
+        fail();
+      } catch (IllegalArgumentException e) {
+        // expected
+      }
+      // 3
+      try {
+        ADMIN.mergeRegionsAsync(
+          tableRegions.stream().map(RegionInfo::getEncodedNameAsBytes).toArray(byte[][]::new),
+          false).get();
+        fail();
+      } catch (DoNotRetryIOException e) {
+        // expected
+      }
+    } finally {
+      ADMIN.disableTable(tableName);
+      ADMIN.deleteTable(tableName);
+    }
+  }
+
+  @Test
   public void testSplitShouldNotHappenIfSplitIsDisabledForTable()
       throws Exception {
     final TableName tableName = TableName.valueOf(name.getMethodName());
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi2.java
index 1642eec..bb0b813 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi2.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi2.java
@@ -17,7 +17,22 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME;
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
 import org.apache.hadoop.hbase.AsyncMetaTableAccessor;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionLocation;
@@ -33,15 +48,6 @@ import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Optional;
-
-import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
 /**
  * Class to test asynchronous region admin operations.
  * @see TestAsyncRegionAdminApi This test and it used to be joined it was taking longer than our
@@ -178,6 +184,40 @@ public class TestAsyncRegionAdminApi2 extends TestAsyncAdminBase {
   }
 
   @Test
+  public void testMergeRegionsInvalidRegionCount() throws InterruptedException {
+    byte[][] splitRows = new byte[][] { Bytes.toBytes("3"), Bytes.toBytes("6") };
+    createTableWithDefaultConf(tableName, splitRows);
+    List<RegionInfo> regions = admin.getRegions(tableName).join();
+    // 0
+    try {
+      admin.mergeRegions(Collections.emptyList(), false).get();
+      fail();
+    } catch (ExecutionException e) {
+      // expected
+      assertThat(e.getCause(), instanceOf(IllegalArgumentException.class));
+    }
+    // 1
+    try {
+      admin.mergeRegions(regions.stream().limit(1).map(RegionInfo::getEncodedNameAsBytes)
+        .collect(Collectors.toList()), false).get();
+      fail();
+    } catch (ExecutionException e) {
+      // expected
+      assertThat(e.getCause(), instanceOf(IllegalArgumentException.class));
+    }
+    // 3
+    try {
+      admin.mergeRegions(
+        regions.stream().map(RegionInfo::getEncodedNameAsBytes).collect(Collectors.toList()), false)
+        .get();
+      fail();
+    } catch (ExecutionException e) {
+      // expected
+      assertThat(e.getCause(), instanceOf(DoNotRetryIOException.class));
+    }
+  }
+
+  @Test
   public void testSplitTable() throws Exception {
     initSplitMergeSwitch();
     splitTest(TableName.valueOf("testSplitTable"), 3000, false, null);


[hbase] 31/49: HBASE-21619. Fix warning message caused by incorrect ternary operator evaluation

Posted by op...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

openinx pushed a commit to branch HBASE-21879
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit 33f344449f286f872af15d7b1d65e9fa481d29ce
Author: Wei-Chiu Chuang <we...@cloudera.com>
AuthorDate: Mon Mar 25 19:58:12 2019 +0100

    HBASE-21619. Fix warning message caused by incorrect ternary operator evaluation
    
    Signed-off-by: Peter Somogyi <ps...@apache.org>
---
 .../java/org/apache/hadoop/hbase/tool/LoadIncrementalHFiles.java   | 7 +++----
 1 file changed, 3 insertions(+), 4 deletions(-)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/LoadIncrementalHFiles.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/LoadIncrementalHFiles.java
index 82f5da4..ec349fe 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/LoadIncrementalHFiles.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/LoadIncrementalHFiles.java
@@ -330,10 +330,9 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
 
       if (queue.isEmpty()) {
         LOG.warn(
-          "Bulk load operation did not find any files to load in " + "directory " + hfofDir != null
-              ? hfofDir.toUri().toString()
-              : "" + ".  Does it contain files in " +
-                  "subdirectories that correspond to column family names?");
+            "Bulk load operation did not find any files to load in directory {}. " +
+            "Does it contain files in subdirectories that correspond to column family names?",
+            (hfofDir != null ? hfofDir.toUri().toString() : ""));
         return Collections.emptyMap();
       }
       pool = createExecutorService();


[hbase] 25/49: HBASE-22095 Taking a snapshot fails in local mode

Posted by op...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

openinx pushed a commit to branch HBASE-21879
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit 2e57be16c7db4bfbda82a419bb3ea4fcc23f4339
Author: Toshihiro Suzuki <br...@gmail.com>
AuthorDate: Sun Mar 24 00:01:56 2019 +0900

    HBASE-22095 Taking a snapshot fails in local mode
    
    Signed-off-by: Zach York <zy...@apache.org>
---
 .../hbase/master/snapshot/TakeSnapshotHandler.java |  2 +-
 .../hbase/snapshot/SnapshotDescriptionUtils.java   |  6 ++-
 .../snapshot/TestSnapshotDescriptionUtils.java     | 50 ++++++++++++++++------
 3 files changed, 42 insertions(+), 16 deletions(-)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java
index 3b7d65a..0d65264 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java
@@ -277,6 +277,7 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh
     URI workingURI = workingDirFs.getUri();
     URI rootURI = fs.getUri();
     if ((!workingURI.getScheme().equals(rootURI.getScheme()) ||
+        workingURI.getAuthority() == null ||
         !workingURI.getAuthority().equals(rootURI.getAuthority()) ||
         workingURI.getUserInfo() == null ||
         !workingURI.getUserInfo().equals(rootURI.getUserInfo()) ||
@@ -365,5 +366,4 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh
   public ForeignException getException() {
     return monitor.getException();
   }
-
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotDescriptionUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotDescriptionUtils.java
index 39202c4..203a58b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotDescriptionUtils.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotDescriptionUtils.java
@@ -271,9 +271,11 @@ public final class SnapshotDescriptionUtils {
    * @param conf configuration for the HBase cluster
    * @return true if the given workingDir is a subdirectory of the default working directory for
    *   snapshots, false otherwise
+   * @throws IOException if we can't get the root dir
    */
-  public static boolean isWithinDefaultWorkingDir(final Path workingDir, Configuration conf) {
-    Path defaultWorkingDir = getDefaultWorkingSnapshotDir(new Path(conf.get(HConstants.HBASE_DIR)));
+  public static boolean isWithinDefaultWorkingDir(final Path workingDir, Configuration conf)
+    throws IOException {
+    Path defaultWorkingDir = getDefaultWorkingSnapshotDir(FSUtils.getRootDir(conf));
     return workingDir.equals(defaultWorkingDir) || isSubDirectoryOf(workingDir, defaultWorkingDir);
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotDescriptionUtils.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotDescriptionUtils.java
index 81568bb..e2e52b4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotDescriptionUtils.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotDescriptionUtils.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -136,30 +137,53 @@ public class TestSnapshotDescriptionUtils {
   }
 
   @Test
-  public void testIsWithinWorkingDir() {
+  public void testIsWithinWorkingDir() throws IOException {
     Configuration conf = new Configuration();
-    conf.set(HConstants.HBASE_DIR, "hdfs://root/");
+    conf.set(HConstants.HBASE_DIR, "hdfs://localhost/root/");
 
     assertFalse(SnapshotDescriptionUtils.isWithinDefaultWorkingDir(
-        new Path("hdfs://root/"), conf));
+        new Path("hdfs://localhost/root/"), conf));
     assertFalse(SnapshotDescriptionUtils.isWithinDefaultWorkingDir(
-        new Path("hdfs://root/.hbase-snapshotdir"), conf));
+        new Path("hdfs://localhost/root/.hbase-snapshotdir"), conf));
     assertFalse(SnapshotDescriptionUtils.isWithinDefaultWorkingDir(
-        new Path("hdfs://root/.hbase-snapshot"), conf));
+        new Path("hdfs://localhost/root/.hbase-snapshot"), conf));
     assertFalse(SnapshotDescriptionUtils.isWithinDefaultWorkingDir(
-        new Path("hdfs://.hbase-snapshot"), conf));
+        new Path("hdfs://localhost/.hbase-snapshot"), conf));
     assertFalse(SnapshotDescriptionUtils.isWithinDefaultWorkingDir(
-        new Path("hdfs://.hbase-snapshot/.tmp"), conf));
-    assertFalse(SnapshotDescriptionUtils.isWithinDefaultWorkingDir(new Path("hdfs://root"), conf));
+        new Path("hdfs://localhost/.hbase-snapshot/.tmp"), conf));
+    assertFalse(SnapshotDescriptionUtils.isWithinDefaultWorkingDir(
+      new Path("hdfs://localhost/root"), conf));
     assertTrue(SnapshotDescriptionUtils.isWithinDefaultWorkingDir(
-        new Path("hdfs://root/.hbase-snapshot/.tmp"), conf));
+        new Path("hdfs://localhost/root/.hbase-snapshot/.tmp"), conf));
     assertTrue(SnapshotDescriptionUtils.isWithinDefaultWorkingDir(
-        new Path("hdfs://root/.hbase-snapshot/.tmp/snapshot"), conf));
+        new Path("hdfs://localhost/root/.hbase-snapshot/.tmp/snapshot"), conf));
+
+    assertFalse(SnapshotDescriptionUtils.isWithinDefaultWorkingDir(
+        new Path("s3://localhost/root/.hbase-snapshot/"), conf));
+    assertFalse(SnapshotDescriptionUtils.isWithinDefaultWorkingDir(
+      new Path("s3://localhost/root"), conf));
+    assertFalse(SnapshotDescriptionUtils.isWithinDefaultWorkingDir(
+        new Path("s3://localhost/root/.hbase-snapshot/.tmp/snapshot"), conf));
+
+    // for local mode
+    conf = HBaseConfiguration.create();
+    String hbsaeDir = conf.get(HConstants.HBASE_DIR);
 
     assertFalse(SnapshotDescriptionUtils.isWithinDefaultWorkingDir(
-        new Path("s3://root/.hbase-snapshot/"), conf));
-    assertFalse(SnapshotDescriptionUtils.isWithinDefaultWorkingDir(new Path("s3://root"), conf));
+      new Path("file:" + hbsaeDir + "/"), conf));
     assertFalse(SnapshotDescriptionUtils.isWithinDefaultWorkingDir(
-        new Path("s3://root/.hbase-snapshot/.tmp/snapshot"), conf));
+      new Path("file:" + hbsaeDir + "/.hbase-snapshotdir"), conf));
+    assertFalse(SnapshotDescriptionUtils.isWithinDefaultWorkingDir(
+      new Path("file:" + hbsaeDir + "/.hbase-snapshot"), conf));
+    assertFalse(SnapshotDescriptionUtils.isWithinDefaultWorkingDir(
+      new Path("file:/.hbase-snapshot"), conf));
+    assertFalse(SnapshotDescriptionUtils.isWithinDefaultWorkingDir(
+      new Path("file:/.hbase-snapshot/.tmp"), conf));
+    assertFalse(SnapshotDescriptionUtils.isWithinDefaultWorkingDir(
+      new Path("file:" + hbsaeDir), conf));
+    assertTrue(SnapshotDescriptionUtils.isWithinDefaultWorkingDir(
+      new Path("file:" + hbsaeDir + "/.hbase-snapshot/.tmp"), conf));
+    assertTrue(SnapshotDescriptionUtils.isWithinDefaultWorkingDir(
+      new Path("file:" + hbsaeDir + "/.hbase-snapshot/.tmp/snapshot"), conf));
   }
 }


[hbase] 30/49: Add 2.0.5 to the downloads page in place of 2.0.4.

Posted by op...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

openinx pushed a commit to branch HBASE-21879
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit 69d261533624a86d634a68475e91bb7105b4f669
Author: stack <st...@apache.org>
AuthorDate: Mon Mar 25 10:36:26 2019 -0400

    Add 2.0.5 to the downloads page in place of 2.0.4.
---
 src/site/xdoc/downloads.xml | 14 +++++++-------
 1 file changed, 7 insertions(+), 7 deletions(-)

diff --git a/src/site/xdoc/downloads.xml b/src/site/xdoc/downloads.xml
index b2e623c..e99055e 100644
--- a/src/site/xdoc/downloads.xml
+++ b/src/site/xdoc/downloads.xml
@@ -66,23 +66,23 @@ under the License.
     </tr>
     <tr>
       <td style="test-align: left">
-        2.0.4
+        2.0.5
       </td>
       <td style="test-align: left">
-        2018/12/02
+        2019/03/25
       </td>
       <td style="test-align: left">
-        <a href="https://apache.org/dist/hbase/2.0.4/compatibility_report_2.0.3vs2.0.4.html">2.0.3 vs 2.0.4</a>
+        <a href="https://dist.apache.org/repos/dist/release/hbase/2.0.5/api_compare_2.0.4_to_2.0.5RC1.html">2.0.4 vs 2.0.5</a>
       </td>
       <td style="test-align: left">
-        <a href="https://apache.org/dist/hbase/2.0.4/CHANGES.md">Changes</a>
+        <a href="https://dist.apache.org/repos/dist/release/hbase/2.0.5/CHANGES.md">Changes</a>
       </td>
       <td style="test-align: left">
-        <a href="https://apache.org/dist/hbase/2.0.4/RELEASENOTES.md">Release Notes</a>
+        <a href="https://dist.apache.org/repos/dist/release/hbase/2.0.5/RELEASENOTES.md">Release Notes</a>
       </td>
       <td style="test-align: left">
-        <a href="https://apache.org/dist/hbase/2.0.4/hbase-2.0.4-src.tar.gz">src</a> (<a href="https://apache.org/dist/hbase/2.0.4/hbase-2.0.4-src.tar.gz.sha512">sha512</a> <a href="https://apache.org/dist/hbase/2.0.4/hbase-2.0.4-src.tar.gz.asc">asc</a>) <br />
-        <a href="https://apache.org/dist/hbase/2.0.4/hbase-2.0.4-bin.tar.gz">bin</a> (<a href="https://apache.org/dist/hbase/2.0.4/hbase-2.0.4-src.tar.gz.sha512">sha512</a> <a href="https://apache.org/dist/hbase/2.0.4/hbase-2.0.4-src.tar.gz.asc">asc</a>)
+        <a href="https://dist.apache.org/repos/dist/release/hbase/2.0.5/hbase-2.0.5-src.tar.gz">src</a> (<a href="https://dist.apache.org/repos/dist/release/hbase/2.0.5/hbase-2.0.5-src.tar.gz.sha512">sha512</a> <a href="https://dist.apache.org/repos/dist/release/hbase/2.0.5/hbase-2.0.5-src.tar.gz.asc">asc</a>) <br />
+        <a href="https://dist.apache.org/repos/dist/release/hbase/2.0.5/hbase-2.0.5-bin.tar.gz">bin</a> (<a href="https://dist.apache.org/repos/dist/release/hbase/2.0.5/hbase-2.0.5-bin.tar.gz.sha512">sha512</a> <a href="https://dist.apache.org/repos/dist/release/hbase/2.0.5/hbase-2.0.5-bin.tar.gz.asc">asc</a>)
       </td>
     </tr>
     <tr>


[hbase] 17/49: HBASE-22061 SplitTableRegionProcedure should hold the lock of its daughter regions

Posted by op...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

openinx pushed a commit to branch HBASE-21879
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit d637e301f0f256131e88b83ce117c0d0ae58c5eb
Author: Jingyun Tian <ti...@gmail.com>
AuthorDate: Tue Mar 19 14:25:04 2019 +0800

    HBASE-22061 SplitTableRegionProcedure should hold the lock of its daughter regions
---
 .../master/assignment/SplitTableRegionProcedure.java | 20 ++++++++++++++++++++
 1 file changed, 20 insertions(+)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
index 8e0dcd3..7b74919 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
@@ -137,6 +137,26 @@ public class SplitTableRegionProcedure
     }
   }
 
+  @Override
+  protected LockState acquireLock(final MasterProcedureEnv env) {
+    if (env.getProcedureScheduler().waitRegions(this, getTableName(), getParentRegion(),
+      daughter_1_RI, daughter_2_RI)) {
+      try {
+        LOG.debug(LockState.LOCK_EVENT_WAIT + " " + env.getProcedureScheduler().dumpLocks());
+      } catch (IOException e) {
+        // Ignore, just for logging
+      }
+      return LockState.LOCK_EVENT_WAIT;
+    }
+    return LockState.LOCK_ACQUIRED;
+  }
+
+  @Override
+  protected void releaseLock(final MasterProcedureEnv env) {
+    env.getProcedureScheduler().wakeRegions(this, getTableName(), getParentRegion(), daughter_1_RI,
+      daughter_2_RI);
+  }
+
   /**
    * Check whether the region is splittable
    * @param env MasterProcedureEnv


[hbase] 13/49: HBASE-21963 Add a script for building and verifying release candidate

Posted by op...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

openinx pushed a commit to branch HBASE-21879
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit 22f9fad4ca01ccd449a823e968ec9a5b019880fd
Author: TAK LON WU <wu...@amazon.com>
AuthorDate: Sat Feb 23 11:43:47 2019 -0800

    HBASE-21963 Add a script for building and verifying release candidate
---
 dev-support/hbase-vote.sh | 183 ++++++++++++++++++++++++++++++++++++++++++++++
 1 file changed, 183 insertions(+)

diff --git a/dev-support/hbase-vote.sh b/dev-support/hbase-vote.sh
new file mode 100755
index 0000000..4bcd680
--- /dev/null
+++ b/dev-support/hbase-vote.sh
@@ -0,0 +1,183 @@
+#!/usr/bin/env bash
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+set -e
+
+usage() {
+  SCRIPT=$(basename "${BASH_SOURCE[@]}")
+
+  cat << __EOF
+hbase-vote. A script for standard vote which verifies the following items
+1. Checksum of sources and binaries
+2. Signature of sources and binaries
+3. Rat check
+4. Built from source
+5. Unit tests
+
+Usage: ${SCRIPT} -s | --source <url> [-k | --key <signature>] [-f | --keys-file-url <url>] [-o | --output-dir </path/to/use>]
+       ${SCRIPT} -h | --help
+
+  -h | --help                   Show this screen.
+  -s | --source '<url>'         A URL pointing to the release candidate sources and binaries
+                                e.g. https://dist.apache.org/repos/dist/dev/hbase/hbase-<version>RC0/
+  -k | --key '<signature>'      A signature of the public key, e.g. 9AD2AE49
+  -f | --keys-file-url '<url>'   the URL of the key file, default is
+                                http://www.apache.org/dist/hbase/KEYS
+  -o | --output-dir '</path>'   directory which has the stdout and stderr of each verification target
+__EOF
+}
+
+while ((${#})); do
+  case "${1}" in
+    -h | --help )
+      usage; exit 0                 ;;
+    -s | --source  )
+      SOURCE_URL="${2}"; shift 2    ;;
+    -k | --key  )
+      SIGNING_KEY="${2}"; shift 2   ;;
+    -f | --keys-file-url )
+      KEY_FILE_URL="${2}"; shift 2  ;;
+    -o | --output-dir )
+      OUTPUT_DIR="${2}"; shift 2    ;;
+    *           )
+      usage >&2; exit 1             ;;
+  esac
+done
+
+# Source url must be provided
+if [ -z "${SOURCE_URL}" ]; then
+  usage;
+  exit 1
+fi
+
+cat << __EOF
+Although This tool helps verifying HBase RC build and unit tests,
+operator may still consider to verify the following manually
+1. Verify the compat-check-report.html
+2. Any on cluster Integration test or performance test, e.g. LTT load 100M or
+   ITBLL 1B rows with serverKilling monkey
+3. Other concerns if any
+__EOF
+
+HBASE_RC_VERSION=$(tr "/" "\n" <<< "${SOURCE_URL}" | tail -n2)
+HBASE_VERSION=$(echo "${HBASE_RC_VERSION}" | sed -e 's/RC[0-9]//g' | sed -e 's/hbase-//g')
+JAVA_VERSION=$(java -version 2>&1 | cut -f3 -d' ' | head -n1 | sed -e 's/"//g')
+OUTPUT_DIR="${OUTPUT_DIR:-$(pwd)}"
+
+if [ ! -d "${OUTPUT_DIR}" ]; then
+    echo "Output directory ${OUTPUT_DIR} does not exist, please create it before running this script."
+    exit 1
+fi
+
+OUTPUT_PATH_PREFIX="${OUTPUT_DIR}"/"${HBASE_RC_VERSION}"
+
+# default value for verification targets, 0 = failed
+SIGNATURE_PASSED=0
+CHECKSUM_PASSED=0
+RAT_CHECK_PASSED=0
+BUILD_FROM_SOURCE_PASSED=0
+UNIT_TEST_PASSED=0
+
+function download_and_import_keys() {
+    KEY_FILE_URL="${KEY_FILE_URL:-https://www.apache.org/dist/hbase/KEYS}"
+    echo "Obtain and import the publisher key(s) from ${KEY_FILE_URL}"
+    # download the keys file into file KEYS
+    wget -O KEYS "${KEY_FILE_URL}"
+    gpg --import KEYS
+    if [ -n "${SIGNING_KEY}" ]; then
+        gpg --list-keys "${SIGNING_KEY}"
+    fi
+}
+
+function download_release_candidate () {
+    # get all files from release candidate repo
+    wget --recursive --no-parent "${SOURCE_URL}"
+}
+
+function verify_signatures() {
+    rm -f "${OUTPUT_PATH_PREFIX}"_verify_signatures
+    for file in *.tar.gz; do
+        gpg --verify "${file}".asc "${file}" >> "${OUTPUT_PATH_PREFIX}"_verify_signatures 2>&1 && SIGNATURE_PASSED=1 || SIGNATURE_PASSED=0
+    done
+}
+
+function verify_checksums() {
+    rm -f "${OUTPUT_PATH_PREFIX}"_verify_checksums
+    SHA_EXT=$(find . -name "*.sha*" | awk -F '.' '{ print $NF }' | head -n 1)
+    for file in *.tar.gz; do
+        gpg --print-md SHA512 "${file}" > "${file}"."${SHA_EXT}".tmp
+        diff "${file}"."${SHA_EXT}".tmp "${file}"."${SHA_EXT}" >> "${OUTPUT_PATH_PREFIX}"_verify_checksums 2>&1 && CHECKSUM_PASSED=1 || CHECKSUM_PASSED=0
+        rm -f "${file}"."${SHA_EXT}".tmp
+    done
+}
+
+function unzip_from_source() {
+    tar -zxvf hbase-"${HBASE_VERSION}"-src.tar.gz
+    cd hbase-"${HBASE_VERSION}"
+}
+
+function rat_test() {
+    rm -f "${OUTPUT_PATH_PREFIX}"_rat_test
+    mvn clean apache-rat:check > "${OUTPUT_PATH_PREFIX}"_rat_test 2>&1 && RAT_CHECK_PASSED=1
+}
+
+function build_from_source() {
+    rm -f "${OUTPUT_PATH_PREFIX}"_build_from_source
+    mvn clean install -DskipTests > "${OUTPUT_PATH_PREFIX}"_build_from_source 2>&1 && BUILD_FROM_SOURCE_PASSED=1
+}
+
+function run_all_tests() {
+    rm -f "${OUTPUT_PATH_PREFIX}"_run_all_tests
+    mvn test -P runAllTests > "${OUTPUT_PATH_PREFIX}"_run_all_tests 2>&1 && UNIT_TEST_PASSED=1
+}
+
+function execute() {
+   ${1} || print_when_exit
+}
+
+function print_when_exit() {
+  cat << __EOF
+        * Signature: $( ((SIGNATURE_PASSED)) && echo "ok" || echo "failed" )
+        * Checksum : $( ((CHECKSUM_PASSED)) && echo "ok" || echo "failed" )
+        * Rat check (${JAVA_VERSION}): $( ((RAT_CHECK_PASSED)) && echo "ok" || echo "failed" )
+         - mvn clean apache-rat:check
+        * Built from source (${JAVA_VERSION}): $( ((BUILD_FROM_SOURCE_PASSED)) && echo "ok" || echo "failed" )
+         - mvn clean install -DskipTests
+        * Unit tests pass (${JAVA_VERSION}): $( ((UNIT_TEST_PASSED)) && echo "ok" || echo "failed" )
+         - mvn test -P runAllTests
+__EOF
+  if ((CHECKSUM_PASSED)) && ((SIGNATURE_PASSED)) && ((RAT_CHECK_PASSED)) && ((BUILD_FROM_SOURCE_PASSED)) && ((UNIT_TEST_PASSED)) ; then
+    exit 0
+  fi
+  exit 1
+}
+
+download_and_import_keys
+download_release_candidate
+pushd dist.apache.org/repos/dist/dev/hbase/"${HBASE_RC_VERSION}"
+
+execute verify_signatures
+execute verify_checksums
+execute unzip_from_source
+execute rat_test
+execute build_from_source
+execute run_all_tests
+
+popd
+
+print_when_exit


[hbase] 36/49: Fix the download links for tgzs for 2.0.5

Posted by op...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

openinx pushed a commit to branch HBASE-21879
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit 116a62d390db1997d76302cb67e58e2cdefe8a2d
Author: stack <st...@apache.org>
AuthorDate: Tue Mar 26 13:01:22 2019 -0400

    Fix the download links for tgzs for 2.0.5
---
 src/site/xdoc/downloads.xml | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/src/site/xdoc/downloads.xml b/src/site/xdoc/downloads.xml
index 365fe9c..38f17e2 100644
--- a/src/site/xdoc/downloads.xml
+++ b/src/site/xdoc/downloads.xml
@@ -81,8 +81,8 @@ under the License.
         <a href="https://dist.apache.org/repos/dist/release/hbase/2.0.5/RELEASENOTES.md">Release Notes</a>
       </td>
       <td style="test-align: left">
-        <a href="https://dist.apache.org/repos/dist/release/hbase/2.0.5/hbase-2.0.5-src.tar.gz">src</a> (<a href="https://dist.apache.org/repos/dist/release/hbase/2.0.5/hbase-2.0.5-src.tar.gz.sha512">sha512</a> <a href="https://dist.apache.org/repos/dist/release/hbase/2.0.5/hbase-2.0.5-src.tar.gz.asc">asc</a>) <br />
-        <a href="https://dist.apache.org/repos/dist/release/hbase/2.0.5/hbase-2.0.5-bin.tar.gz">bin</a> (<a href="https://dist.apache.org/repos/dist/release/hbase/2.0.5/hbase-2.0.5-bin.tar.gz.sha512">sha512</a> <a href="https://dist.apache.org/repos/dist/release/hbase/2.0.5/hbase-2.0.5-bin.tar.gz.asc">asc</a>)
+        <a href="https://www.apache.org/dyn/closer.lua/hbase/2.0.5/hbase-2.0.5-src.tar.gz">src</a> (<a href="https://dist.apache.org/repos/dist/release/hbase/2.0.5/hbase-2.0.5-src.tar.gz.sha512">sha512</a> <a href="https://dist.apache.org/repos/dist/release/hbase/2.0.5/hbase-2.0.5-src.tar.gz.asc">asc</a>) <br />
+        <a href="https://www.apache.org/dyn/closer.lua/hbase/2.0.5/hbase-2.0.5-bin.tar.gz">bin</a> (<a href="https://dist.apache.org/repos/dist/release/hbase/2.0.5/hbase-2.0.5-bin.tar.gz.sha512">sha512</a> <a href="https://dist.apache.org/repos/dist/release/hbase/2.0.5/hbase-2.0.5-bin.tar.gz.asc">asc</a>)
       </td>
     </tr>
     <tr>


[hbase] 20/49: HBASE-22065 Add listTableDescriptors(List) method in AsyncAdmin

Posted by op...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

openinx pushed a commit to branch HBASE-21879
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit 1515fe11ba286475958162220c13fed4dcda216b
Author: niuyulin <ni...@xiaomi.com>
AuthorDate: Wed Mar 20 15:27:59 2019 +0800

    HBASE-22065 Add listTableDescriptors(List<TableName>) method in AsyncAdmin
    
    Signed-off-by: zhangduo <zh...@apache.org>
---
 .../org/apache/hadoop/hbase/client/AsyncAdmin.java     |  7 +++++++
 .../apache/hadoop/hbase/client/AsyncHBaseAdmin.java    |  5 +++++
 .../apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java | 10 ++++++++++
 .../hadoop/hbase/client/TestAsyncTableAdminApi3.java   | 18 ++++++++++++++++++
 4 files changed, 40 insertions(+)

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 07807ed..f7adc16 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
@@ -88,6 +88,13 @@ public interface AsyncAdmin {
       boolean includeSysTables);
 
   /**
+   * List specific tables including system tables.
+   * @param tableNames the table list to match against
+   * @return - returns a list of TableDescriptors wrapped by a {@link CompletableFuture}.
+   */
+  CompletableFuture<List<TableDescriptor>> listTableDescriptors(List<TableName> tableNames);
+
+  /**
    * Get list of table descriptors by namespace.
    * @param name namespace name
    * @return returns a list of TableDescriptors wrapped by a {@link CompletableFuture}.
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 038f94e..0eceaad 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
@@ -89,6 +89,11 @@ class AsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
+  public CompletableFuture<List<TableDescriptor>> listTableDescriptors(List<TableName> tableNames) {
+    return wrap(rawAdmin.listTableDescriptors(tableNames));
+  }
+
+  @Override
   public CompletableFuture<List<TableDescriptor>> listTableDescriptorsByNamespace(String name) {
     return wrap(rawAdmin.listTableDescriptorsByNamespace(name));
   }
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 498ef49..587c6e2 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
@@ -473,6 +473,16 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
       includeSysTables));
   }
 
+  @Override
+  public CompletableFuture<List<TableDescriptor>> listTableDescriptors(List<TableName> tableNames) {
+    Preconditions.checkNotNull(tableNames,
+      "tableNames is null. If you don't specify tableNames, " + "use listTables(boolean) instead");
+    if (tableNames.isEmpty()) {
+      return CompletableFuture.completedFuture(Collections.emptyList());
+    }
+    return getTableDescriptors(RequestConverter.buildGetTableDescriptorsRequest(tableNames));
+  }
+
   private CompletableFuture<List<TableDescriptor>>
       getTableDescriptors(GetTableDescriptorsRequest request) {
     return this.<List<TableDescriptor>> newMasterCaller()
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi3.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi3.java
index 750749d..d4bb3be 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi3.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi3.java
@@ -25,6 +25,8 @@ import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
+import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 import java.util.Optional;
 import java.util.concurrent.ExecutionException;
@@ -110,6 +112,22 @@ public class TestAsyncTableAdminApi3 extends TestAsyncAdminBase {
       assertTrue("Not found: " + tables[i], found);
     }
 
+    tableNames = new ArrayList<TableName>(tables.length + 1);
+    tableDescs = admin.listTableDescriptors(tableNames).get();
+    size = tableDescs.size();
+    assertEquals(0, size);
+
+    Collections.addAll(tableNames, tables);
+    tableNames.add(TableName.META_TABLE_NAME);
+    tableDescs = admin.listTableDescriptors(tableNames).get();
+    size = tableDescs.size();
+    assertEquals(tables.length + 1, size);
+    for (int i = 0, j = 0; i < tables.length && j < size; i++, j++) {
+      assertTrue("tableName should be equal in order",
+          tableDescs.get(j).getTableName().equals(tables[i]));
+    }
+    assertTrue(tableDescs.get(size - 1).getTableName().equals(TableName.META_TABLE_NAME));
+
     for (int i = 0; i < tables.length; i++) {
       admin.disableTable(tables[i]).join();
       admin.deleteTable(tables[i]).join();


[hbase] 37/49: HBASE-22067 Fix log in StochasticLoadBalancer when balancer is ill-fit for cluster size

Posted by op...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

openinx pushed a commit to branch HBASE-21879
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit 975b2aa8e51825dc99002244f0ca24ee0bc392bb
Author: xcang <xc...@salesforce.com>
AuthorDate: Tue Mar 19 13:42:09 2019 -0700

    HBASE-22067 Fix log in StochasticLoadBalancer when balancer is ill-fit for cluster size
---
 .../org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java
index cbc1a37..7a93c6e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java
@@ -389,7 +389,7 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
             + "maxSteps:{}. Hence load balancing may not work well. Setting parameter "
             + "\"hbase.master.balancer.stochastic.runMaxSteps\" to true can overcome this issue."
             + "(This config change does not require service restart)", calculatedMaxSteps,
-            maxRunningTime);
+            maxSteps);
       }
     }
     LOG.info("start StochasticLoadBalancer.balancer, initCost=" + currentCost + ", functionCost="


[hbase] 16/49: HBASE-22042 Missing @Override annotation for RawAsyncTableImpl.scan

Posted by op...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

openinx pushed a commit to branch HBASE-21879
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit 9063b90a27efbb454d54b2271e8776280051d58f
Author: Rishabh Jain <ri...@gmail.com>
AuthorDate: Thu Mar 14 01:47:15 2019 -0400

    HBASE-22042 Missing @Override annotation for RawAsyncTableImpl.scan
    
    Signed-off-by: zhangduo <zh...@apache.org>
---
 .../src/main/java/org/apache/hadoop/hbase/client/RawAsyncTableImpl.java  | 1 +
 1 file changed, 1 insertion(+)

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 1925c0e..86f11b9 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
@@ -436,6 +436,7 @@ class RawAsyncTableImpl implements AsyncTable<AdvancedScanResultConsumer> {
     return newScan;
   }
 
+  @Override
   public void scan(Scan scan, AdvancedScanResultConsumer consumer) {
     new AsyncClientScanner(setDefaultScanConfig(scan), consumer, tableName, conn, retryTimer,
       pauseNs, maxAttempts, scanTimeoutNs, readRpcTimeoutNs, startLogErrorsCnt).start();


[hbase] 12/49: HBASE-22009 Improve RSGroupInfoManagerImpl#getDefaultServers()

Posted by op...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

openinx pushed a commit to branch HBASE-21879
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit 3c8715aea853e91ac1bf019b353101c27b597679
Author: Xiang Li <li...@freewheel.tv>
AuthorDate: Thu Mar 7 16:20:18 2019 +0000

    HBASE-22009 Improve RSGroupInfoManagerImpl#getDefaultServers()
    
    Signed-off-by: Xu Cang <xu...@apache.org>
---
 .../hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java   | 18 ++++++++++--------
 1 file changed, 10 insertions(+), 8 deletions(-)

diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
index 17d6481..7549626 100644
--- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
+++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
@@ -596,17 +596,19 @@ final class RSGroupInfoManagerImpl implements RSGroupInfoManager {
 
   // Called by ServerEventsListenerThread. Presume it has lock on this manager when it runs.
   private SortedSet<Address> getDefaultServers() throws IOException {
+    // Build a list of servers in other groups than default group, from rsGroupMap
+    Set<Address> serversInOtherGroup = new HashSet<>();
+    for (RSGroupInfo group : listRSGroups() /* get from rsGroupMap */) {
+      if (!RSGroupInfo.DEFAULT_GROUP.equals(group.getName())) { // not default group
+        serversInOtherGroup.addAll(group.getServers());
+      }
+    }
+
+    // Get all online servers from Zookeeper and find out servers in default group
     SortedSet<Address> defaultServers = Sets.newTreeSet();
     for (ServerName serverName : getOnlineRS()) {
       Address server = Address.fromParts(serverName.getHostname(), serverName.getPort());
-      boolean found = false;
-      for (RSGroupInfo rsgi : listRSGroups()) {
-        if (!RSGroupInfo.DEFAULT_GROUP.equals(rsgi.getName()) && rsgi.containsServer(server)) {
-          found = true;
-          break;
-        }
-      }
-      if (!found) {
+      if (!serversInOtherGroup.contains(server)) { // not in other groups
         defaultServers.add(server);
       }
     }


[hbase] 46/49: HBASE-22092 Typo in block cache monitoring documentation

Posted by op...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

openinx pushed a commit to branch HBASE-21879
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit c2bee58c26e269839f71f1f351b8d2d3ae67dfa7
Author: William Shen <wi...@users.noreply.github.com>
AuthorDate: Fri Mar 22 16:08:12 2019 -0700

    HBASE-22092 Typo in block cache monitoring documentation
---
 src/main/asciidoc/_chapters/ops_mgt.adoc | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/src/main/asciidoc/_chapters/ops_mgt.adoc b/src/main/asciidoc/_chapters/ops_mgt.adoc
index b4bdb21..d8d9a4a 100644
--- a/src/main/asciidoc/_chapters/ops_mgt.adoc
+++ b/src/main/asciidoc/_chapters/ops_mgt.adoc
@@ -1806,7 +1806,7 @@ image::bc_config.png[]
 .Stats shows statistics about the performance of the cache.
 image::bc_stats.png[]
 
-.L1 and L2 shows information about the L1 and L2 caches.
+.L1 and L2 show information about the L1 and L2 caches.
 image::bc_l1.png[]
 
 This is not an exhaustive list of all the screens and reports available.


[hbase] 04/49: HBASE-22044 ByteBufferUtils should not be IA.Public API

Posted by op...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

openinx pushed a commit to branch HBASE-21879
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit 0fb208f635fb20e078ea9cbefc3482d194cb3d1a
Author: Sean Busbey <bu...@apache.org>
AuthorDate: Tue Mar 12 13:23:57 2019 -0500

    HBASE-22044 ByteBufferUtils should not be IA.Public API
    
    Signed-off-by: Andrew Purtell <ap...@apache.org>
---
 .../src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java     | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java
index ac7bd30..98bc88a 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java
@@ -43,7 +43,7 @@ import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesti
  * variable-length long numbers.
  */
 @SuppressWarnings("restriction")
-@InterfaceAudience.Public
+@InterfaceAudience.Private
 public final class ByteBufferUtils {
   // "Compressed integer" serialization helper constants.
   public final static int VALUE_MASK = 0x7f;


[hbase] 27/49: Revert "HBASE-21965 Fix failed split and merge transactions that have failed to roll back"

Posted by op...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

openinx pushed a commit to branch HBASE-21879
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit 2169a37a2e84e955a5dd8007b45794c030d0318a
Author: zhangduo <zh...@apache.org>
AuthorDate: Sun Mar 24 13:51:34 2019 +0800

    Revert "HBASE-21965 Fix failed split and merge transactions that have failed to roll back"
    
    This reverts commit 7e3ca7063cc1a4b431e0b07e8db91537252db3c3.
---
 .../org/apache/hadoop/hbase/client/HBaseHbck.java  |  16 --
 .../java/org/apache/hadoop/hbase/client/Hbck.java  |  15 --
 .../hbase/shaded/protobuf/RequestConverter.java    |   9 --
 .../src/main/protobuf/Master.proto                 |  16 --
 .../hadoop/hbase/master/MasterRpcServices.java     | 167 --------------------
 .../assignment/SplitTableRegionProcedure.java      |  66 ++++----
 .../hbase/regionserver/HRegionFileSystem.java      |   6 -
 .../org/apache/hadoop/hbase/client/TestHbck.java   | 172 ---------------------
 8 files changed, 28 insertions(+), 439 deletions(-)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseHbck.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseHbck.java
index 1963906..a276017 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseHbck.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseHbck.java
@@ -19,12 +19,10 @@ package org.apache.hadoop.hbase.client;
 
 import java.io.IOException;
 import java.util.List;
-import java.util.Map;
 import java.util.concurrent.Callable;
 import java.util.stream.Collectors;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
@@ -174,18 +172,4 @@ public class HBaseHbck implements Hbck {
       throw new IOException(se);
     }
   }
-
-  @Override
-  public Map<String, MasterProtos.REGION_ERROR_TYPE>
-      getFailedSplitMergeLegacyRegions(List<TableName> tableNames) throws IOException {
-    try {
-      MasterProtos.GetFailedSplitMergeLegacyRegionsResponse response =
-          this.hbck.getFailedSplitMergeLegacyRegions(rpcControllerFactory.newController(),
-            RequestConverter.toGetFailedSplitMergeLegacyRegionsRequest(tableNames));
-      return response.getErrorsMap();
-    } catch (ServiceException se) {
-      LOG.debug("get failed split/merge legacy regions failed", se);
-      throw new IOException(se);
-    }
-  }
 }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Hbck.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Hbck.java
index 49fb18f..e88805c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Hbck.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Hbck.java
@@ -20,15 +20,12 @@ package org.apache.hadoop.hbase.client;
 import java.io.Closeable;
 import java.io.IOException;
 import java.util.List;
-import java.util.Map;
 
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.hadoop.hbase.TableName;
 import org.apache.yetus.audience.InterfaceAudience;
 
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos;
 
 /**
  * Hbck fixup tool APIs. Obtain an instance from {@link ClusterConnection#getHbck()} and call
@@ -109,16 +106,4 @@ public interface Hbck extends Abortable, Closeable {
 
   List<Long> scheduleServerCrashProcedure(List<HBaseProtos.ServerName> serverNames)
       throws IOException;
-
-  /**
-   * This method is to get the regions which left by failed split/merge procedures for a certain
-   * table. There are two kinds of region this method will return. One is orphan regions left on FS,
-   * which left because split/merge procedure crashed before updating meta. And the other one is
-   * unassigned split daughter region or merged region, which left because split/merge procedure
-   * crashed before assignment.
-   * @param tableName table to check
-   * @return Map of problematic regions
-   */
-  Map<String, MasterProtos.REGION_ERROR_TYPE>
-      getFailedSplitMergeLegacyRegions(List<TableName> tableName) throws IOException;
 }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
index e7b6624..36c8fab 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
@@ -1913,15 +1913,6 @@ public final class RequestConverter {
     return b.addAllServerName(serverNames).build();
   }
 
-  public static MasterProtos.GetFailedSplitMergeLegacyRegionsRequest
-      toGetFailedSplitMergeLegacyRegionsRequest(List<TableName> tableNames) {
-    MasterProtos.GetFailedSplitMergeLegacyRegionsRequest.Builder b =
-        MasterProtos.GetFailedSplitMergeLegacyRegionsRequest.newBuilder();
-    List<HBaseProtos.TableName> protoTableNames = tableNames.stream()
-        .map(tableName -> ProtobufUtil.toProtoTableName(tableName)).collect(Collectors.toList());
-    return b.addAllTable(protoTableNames).build();
-  }
-
   private static List<RegionSpecifier> toEncodedRegionNameRegionSpecifiers(
       List<String> encodedRegionNames) {
     return encodedRegionNames.stream().
diff --git a/hbase-protocol-shaded/src/main/protobuf/Master.proto b/hbase-protocol-shaded/src/main/protobuf/Master.proto
index f9fbf08..4ed0ad5 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Master.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Master.proto
@@ -1093,19 +1093,6 @@ message ScheduleServerCrashProcedureResponse {
   repeated uint64 pid = 1;
 }
 
-message GetFailedSplitMergeLegacyRegionsRequest {
-  repeated TableName table = 1;
-}
-
-enum REGION_ERROR_TYPE {
-  daughter_merged_region_not_online = 0;
-  orphan_region_on_fs = 1;
-}
-
-message GetFailedSplitMergeLegacyRegionsResponse {
-  map<string, REGION_ERROR_TYPE> errors = 1;
-}
-
 service HbckService {
   /** Update state of the table in meta only*/
   rpc SetTableStateInMeta(SetTableStateInMetaRequest)
@@ -1136,7 +1123,4 @@ service HbckService {
   /** Schedule a ServerCrashProcedure to help recover a crash server */
   rpc ScheduleServerCrashProcedure(ScheduleServerCrashProcedureRequest)
     returns(ScheduleServerCrashProcedureResponse);
-
-  rpc getFailedSplitMergeLegacyRegions(GetFailedSplitMergeLegacyRegionsRequest)
-    returns(GetFailedSplitMergeLegacyRegionsResponse);
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index c396036..b943000 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -27,14 +27,12 @@ import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashSet;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
 import java.util.stream.Collectors;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.ClusterMetricsBuilder;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
@@ -51,7 +49,6 @@ import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.MasterSwitchType;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionInfoBuilder;
-import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableState;
@@ -69,10 +66,7 @@ import org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface;
 import org.apache.hadoop.hbase.ipc.RpcServerFactory;
 import org.apache.hadoop.hbase.ipc.RpcServerInterface;
 import org.apache.hadoop.hbase.ipc.ServerRpcController;
-import org.apache.hadoop.hbase.master.assignment.MergeTableRegionsProcedure;
-import org.apache.hadoop.hbase.master.assignment.RegionStateStore;
 import org.apache.hadoop.hbase.master.assignment.RegionStates;
-import org.apache.hadoop.hbase.master.assignment.SplitTableRegionProcedure;
 import org.apache.hadoop.hbase.master.locking.LockProcedure;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil;
@@ -92,7 +86,6 @@ import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
 import org.apache.hadoop.hbase.quotas.QuotaObserverChore;
 import org.apache.hadoop.hbase.quotas.QuotaUtil;
 import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot;
-import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
 import org.apache.hadoop.hbase.regionserver.RSRpcServices;
 import org.apache.hadoop.hbase.regionserver.RpcSchedulerFactory;
 import org.apache.hadoop.hbase.replication.ReplicationException;
@@ -110,10 +103,8 @@ import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
 import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
 import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.hbase.util.PairOfSameType;
 import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -2504,164 +2495,6 @@ public class MasterRpcServices extends RSRpcServices
   }
 
   @Override
-  public MasterProtos.GetFailedSplitMergeLegacyRegionsResponse getFailedSplitMergeLegacyRegions(
-      RpcController controller, MasterProtos.GetFailedSplitMergeLegacyRegionsRequest request)
-      throws ServiceException {
-    List<HBaseProtos.TableName> tables = request.getTableList();
-
-    Map<String, MasterProtos.REGION_ERROR_TYPE> errorRegions = new HashMap<>();
-    try {
-      for (HBaseProtos.TableName tableName : tables) {
-        errorRegions.putAll(getFailedSplitMergeLegacyRegions(ProtobufUtil.toTableName(tableName)));
-      }
-    } catch (IOException e) {
-      throw new ServiceException(e);
-    }
-    return MasterProtos.GetFailedSplitMergeLegacyRegionsResponse.newBuilder()
-        .putAllErrors(errorRegions).build();
-  }
-
-  private Map<String, MasterProtos.REGION_ERROR_TYPE>
-      getFailedSplitMergeLegacyRegions(TableName tableName) throws IOException {
-    if (!MetaTableAccessor.tableExists(master.getConnection(), tableName)) {
-      throw new IOException("table " + tableName.getNameAsString() + " doesn't exist");
-    }
-    if (!MetaTableAccessor.getTableState(master.getConnection(), tableName).isEnabled()) {
-      throw new IOException(
-          "table " + tableName.getNameAsString() + " is not enabled yet");
-    }
-    final Map<String, MasterProtos.REGION_ERROR_TYPE> problemRegions = new HashMap<>();
-
-    // Case 1. find orphan region on fs
-    // orphan regions may due to a failed split region procedure, which daughter regions are created
-    // then the procedure is aborted. Or merged region is created then the procedure is aborted.
-    List<String> orphanRegions = findOrphanRegionOnFS(tableName);
-    orphanRegions.stream().forEach(
-      region -> problemRegions.put(region, MasterProtos.REGION_ERROR_TYPE.orphan_region_on_fs));
-
-    // Case 2. find unassigned daughter regions or merged regions
-    List<String> unassignedDaughterOrMergedRegions =
-        findUnassignedDaughterOrMergedRegions(tableName);
-    unassignedDaughterOrMergedRegions.stream().forEach(region -> problemRegions.put(region,
-      MasterProtos.REGION_ERROR_TYPE.daughter_merged_region_not_online));
-
-    // if these regions in problemRegions are currently handled by SplitTableRegionProcedure or
-    // MergeTableRegionsProcedure, we should remove them from this map
-    master.getProcedures().stream().filter(p -> !(p.isFinished() || p.isBypass())).forEach(p -> {
-      if (p instanceof SplitTableRegionProcedure) {
-        problemRegions
-            .remove(((SplitTableRegionProcedure) p).getDaughterOneRI().getRegionNameAsString());
-        problemRegions
-            .remove(((SplitTableRegionProcedure) p).getDaughterTwoRI().getRegionNameAsString());
-      } else if (p instanceof MergeTableRegionsProcedure) {
-        problemRegions
-            .remove(((MergeTableRegionsProcedure) p).getMergedRegion().getRegionNameAsString());
-      }
-    });
-
-    // check if regions are still problematic now
-    checkRegionStillProblematic(problemRegions, tableName);
-    return problemRegions;
-  }
-
-
-  private void checkRegionStillProblematic(
-      Map<String, MasterProtos.REGION_ERROR_TYPE> problemRegions, TableName tableName)
-      throws IOException {
-    Iterator<Map.Entry<String, MasterProtos.REGION_ERROR_TYPE>> iterator =
-        problemRegions.entrySet().iterator();
-    while (iterator.hasNext()) {
-      Map.Entry<String, MasterProtos.REGION_ERROR_TYPE> entry = iterator.next();
-      Result r = MetaTableAccessor.getRegionResult(master.getConnection(),
-        Bytes.toBytesBinary(entry.getKey()));
-      RegionInfo regionInfo = MetaTableAccessor.getRegionInfo(r);
-      switch (entry.getValue()) {
-        case orphan_region_on_fs:
-          // region is build for this directory, it is not a problematic region any more
-          if (r != null) {
-            problemRegions.remove(regionInfo.getRegionNameAsString());
-          }
-          break;
-        case daughter_merged_region_not_online:
-          RegionState.State state = RegionStateStore.getRegionState(r, 0);
-          if (!state.matches(RegionState.State.CLOSED, RegionState.State.SPLITTING_NEW,
-            RegionState.State.MERGED)) {
-            problemRegions.remove(regionInfo.getRegionNameAsString());
-          }
-          break;
-        default:
-          throw new IOException("there should be no problematic region of this type");
-      }
-    }
-  }
-
-  private List<String> findUnassignedDaughterOrMergedRegions(TableName tableName)
-      throws IOException {
-    Set<String> checkRegions = new HashSet<>();
-    Map<String, RegionState.State> regionStates = new HashMap<>();
-    Map<String, RegionInfo> regionInfos = new HashMap<>();
-
-    MetaTableAccessor.scanMeta(master.getConnection(), tableName,
-      MetaTableAccessor.QueryType.REGION, Integer.MAX_VALUE, r -> {
-        RegionInfo regionInfo = MetaTableAccessor.getRegionInfo(r);
-        regionInfos.put(regionInfo.getRegionNameAsString(), regionInfo);
-        RegionState.State state = RegionStateStore.getRegionState(r, 0);
-        regionStates.put(regionInfo.getEncodedName(), state);
-        if (regionInfo.isSplitParent()) {
-          PairOfSameType<RegionInfo> daughters = MetaTableAccessor.getDaughterRegions(r);
-          checkRegions.add(daughters.getFirst().getRegionNameAsString());
-          checkRegions.add(daughters.getSecond().getRegionNameAsString());
-        } else if (r.getValue(HConstants.CATALOG_FAMILY, HConstants.MERGEA_QUALIFIER) != null
-            || r.getValue(HConstants.CATALOG_FAMILY, HConstants.MERGEB_QUALIFIER) != null) {
-          checkRegions.add(regionInfo.getRegionNameAsString());
-        }
-        return true;
-      });
-
-    // find unassigned merged or split daughter region
-    return checkRegions.stream().map(regionName -> regionInfos.get(regionName))
-        .filter(regionInfo -> !regionInfo.isSplitParent())
-        .filter(regionInfo -> !regionStates.get(regionInfo.getEncodedName())
-            .matches(RegionState.State.OPEN))
-        .map(regionInfo -> regionInfo.getRegionNameAsString()).collect(Collectors.toList());
-  }
-
-  private List<String> findOrphanRegionOnFS(TableName tableName) throws IOException {
-    // get available regions from meta, merged region should be consider available
-    HashSet<String> regionsInMeta = new HashSet<>();
-    MetaTableAccessor.scanMeta(master.getConnection(), tableName,
-      MetaTableAccessor.QueryType.REGION, Integer.MAX_VALUE, r -> {
-        RegionInfo regionInfo = MetaTableAccessor.getRegionInfo(r);
-        regionsInMeta.add(regionInfo.getEncodedName());
-        if (r.getValue(HConstants.CATALOG_FAMILY, HConstants.MERGEA_QUALIFIER) != null
-            || r.getValue(HConstants.CATALOG_FAMILY, HConstants.MERGEB_QUALIFIER) != null) {
-          PairOfSameType<RegionInfo> mergedRegions = MetaTableAccessor.getMergeRegions(r);
-          regionsInMeta.add(mergedRegions.getFirst().getEncodedName());
-          regionsInMeta.add(mergedRegions.getSecond().getEncodedName());
-        }
-        return true;
-      });
-    // get regionInfo from fs
-    Path tableDir = FSUtils.getTableDir(master.getMasterFileSystem().getRootDir(), tableName);
-    FileStatus[] regions =
-        master.getFileSystem().listStatus(tableDir, path -> !path.getName().startsWith("."));
-    HashMap<String, String> regionNames = new HashMap<>();
-    for (FileStatus region : regions) {
-      RegionInfo regionInfo =
-          HRegionFileSystem.loadRegionInfoFileContent(master.getFileSystem(), region.getPath());
-      regionNames.put(regionInfo.getEncodedName(), regionInfo.getRegionNameAsString());
-    }
-    Iterator<Map.Entry<String, String>> regionIterator = regionNames.entrySet().iterator();
-    while (regionIterator.hasNext()) {
-      Map.Entry<String, String> region = regionIterator.next();
-      if (regionsInMeta.contains(region.getKey())) {
-        regionIterator.remove();
-      }
-    }
-    return new ArrayList<>(regionNames.values());
-  }
-
-  @Override
   public SwitchRpcThrottleResponse switchRpcThrottle(RpcController controller,
       SwitchRpcThrottleRequest request) throws ServiceException {
     try {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
index 657f397..7b74919 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
@@ -93,8 +93,8 @@ public class SplitTableRegionProcedure
     extends AbstractStateMachineRegionProcedure<SplitTableRegionState> {
   private static final Logger LOG = LoggerFactory.getLogger(SplitTableRegionProcedure.class);
   private Boolean traceEnabled = null;
-  private RegionInfo daughterOneRI;
-  private RegionInfo daughterTwoRI;
+  private RegionInfo daughter_1_RI;
+  private RegionInfo daughter_2_RI;
   private byte[] bestSplitRow;
   private RegionSplitPolicy splitPolicy;
 
@@ -113,13 +113,13 @@ public class SplitTableRegionProcedure
     checkSplittable(env, regionToSplit, bestSplitRow);
     final TableName table = regionToSplit.getTable();
     final long rid = getDaughterRegionIdTimestamp(regionToSplit);
-    this.daughterOneRI = RegionInfoBuilder.newBuilder(table)
+    this.daughter_1_RI = RegionInfoBuilder.newBuilder(table)
         .setStartKey(regionToSplit.getStartKey())
         .setEndKey(bestSplitRow)
         .setSplit(false)
         .setRegionId(rid)
         .build();
-    this.daughterTwoRI = RegionInfoBuilder.newBuilder(table)
+    this.daughter_2_RI = RegionInfoBuilder.newBuilder(table)
         .setStartKey(bestSplitRow)
         .setEndKey(regionToSplit.getEndKey())
         .setSplit(false)
@@ -140,7 +140,7 @@ public class SplitTableRegionProcedure
   @Override
   protected LockState acquireLock(final MasterProcedureEnv env) {
     if (env.getProcedureScheduler().waitRegions(this, getTableName(), getParentRegion(),
-      daughterOneRI, daughterTwoRI)) {
+      daughter_1_RI, daughter_2_RI)) {
       try {
         LOG.debug(LockState.LOCK_EVENT_WAIT + " " + env.getProcedureScheduler().dumpLocks());
       } catch (IOException e) {
@@ -153,18 +153,8 @@ public class SplitTableRegionProcedure
 
   @Override
   protected void releaseLock(final MasterProcedureEnv env) {
-    env.getProcedureScheduler().wakeRegions(this, getTableName(), getParentRegion(), daughterOneRI,
-      daughterTwoRI);
-  }
-
-  @VisibleForTesting
-  public RegionInfo getDaughterOneRI() {
-    return daughterOneRI;
-  }
-
-  @VisibleForTesting
-  public RegionInfo getDaughterTwoRI() {
-    return daughterTwoRI;
+    env.getProcedureScheduler().wakeRegions(this, getTableName(), getParentRegion(), daughter_1_RI,
+      daughter_2_RI);
   }
 
   /**
@@ -426,8 +416,8 @@ public class SplitTableRegionProcedure
         MasterProcedureProtos.SplitTableRegionStateData.newBuilder()
         .setUserInfo(MasterProcedureUtil.toProtoUserInfo(getUser()))
         .setParentRegionInfo(ProtobufUtil.toRegionInfo(getRegion()))
-        .addChildRegionInfo(ProtobufUtil.toRegionInfo(daughterOneRI))
-        .addChildRegionInfo(ProtobufUtil.toRegionInfo(daughterTwoRI));
+        .addChildRegionInfo(ProtobufUtil.toRegionInfo(daughter_1_RI))
+        .addChildRegionInfo(ProtobufUtil.toRegionInfo(daughter_2_RI));
     serializer.serialize(splitTableRegionMsg.build());
   }
 
@@ -441,8 +431,8 @@ public class SplitTableRegionProcedure
     setUser(MasterProcedureUtil.toUserInfo(splitTableRegionsMsg.getUserInfo()));
     setRegion(ProtobufUtil.toRegionInfo(splitTableRegionsMsg.getParentRegionInfo()));
     assert(splitTableRegionsMsg.getChildRegionInfoCount() == 2);
-    daughterOneRI = ProtobufUtil.toRegionInfo(splitTableRegionsMsg.getChildRegionInfo(0));
-    daughterTwoRI = ProtobufUtil.toRegionInfo(splitTableRegionsMsg.getChildRegionInfo(1));
+    daughter_1_RI = ProtobufUtil.toRegionInfo(splitTableRegionsMsg.getChildRegionInfo(0));
+    daughter_2_RI = ProtobufUtil.toRegionInfo(splitTableRegionsMsg.getChildRegionInfo(1));
   }
 
   @Override
@@ -453,9 +443,9 @@ public class SplitTableRegionProcedure
     sb.append(", parent=");
     sb.append(getParentRegion().getShortNameToLog());
     sb.append(", daughterA=");
-    sb.append(daughterOneRI.getShortNameToLog());
+    sb.append(daughter_1_RI.getShortNameToLog());
     sb.append(", daughterB=");
-    sb.append(daughterTwoRI.getShortNameToLog());
+    sb.append(daughter_2_RI.getShortNameToLog());
   }
 
   private RegionInfo getParentRegion() {
@@ -473,7 +463,7 @@ public class SplitTableRegionProcedure
   }
 
   private byte[] getSplitRow() {
-    return daughterTwoRI.getStartKey();
+    return daughter_2_RI.getStartKey();
   }
 
   private static final State[] EXPECTED_SPLIT_STATES = new State[] { State.OPEN, State.CLOSED };
@@ -605,17 +595,17 @@ public class SplitTableRegionProcedure
     Pair<Integer, Integer> expectedReferences = splitStoreFiles(env, regionFs);
 
     assertReferenceFileCount(fs, expectedReferences.getFirst(),
-      regionFs.getSplitsDir(daughterOneRI));
+      regionFs.getSplitsDir(daughter_1_RI));
     //Move the files from the temporary .splits to the final /table/region directory
-    regionFs.commitDaughterRegion(daughterOneRI);
+    regionFs.commitDaughterRegion(daughter_1_RI);
     assertReferenceFileCount(fs, expectedReferences.getFirst(),
-      new Path(tabledir, daughterOneRI.getEncodedName()));
+      new Path(tabledir, daughter_1_RI.getEncodedName()));
 
     assertReferenceFileCount(fs, expectedReferences.getSecond(),
-      regionFs.getSplitsDir(daughterTwoRI));
-    regionFs.commitDaughterRegion(daughterTwoRI);
+      regionFs.getSplitsDir(daughter_2_RI));
+    regionFs.commitDaughterRegion(daughter_2_RI);
     assertReferenceFileCount(fs, expectedReferences.getSecond(),
-      new Path(tabledir, daughterTwoRI.getEncodedName()));
+      new Path(tabledir, daughter_2_RI.getEncodedName()));
   }
 
   /**
@@ -755,9 +745,9 @@ public class SplitTableRegionProcedure
 
     final byte[] splitRow = getSplitRow();
     final String familyName = Bytes.toString(family);
-    final Path path_first = regionFs.splitStoreFile(this.daughterOneRI, familyName, sf, splitRow,
+    final Path path_first = regionFs.splitStoreFile(this.daughter_1_RI, familyName, sf, splitRow,
         false, splitPolicy);
-    final Path path_second = regionFs.splitStoreFile(this.daughterTwoRI, familyName, sf, splitRow,
+    final Path path_second = regionFs.splitStoreFile(this.daughter_2_RI, familyName, sf, splitRow,
        true, splitPolicy);
     if (LOG.isDebugEnabled()) {
       LOG.debug("pid=" + getProcId() + " splitting complete for store file: " +
@@ -822,7 +812,7 @@ public class SplitTableRegionProcedure
    */
   private void updateMeta(final MasterProcedureEnv env) throws IOException {
     env.getAssignmentManager().markRegionAsSplit(getParentRegion(), getParentRegionServerName(env),
-        daughterOneRI, daughterTwoRI);
+      daughter_1_RI, daughter_2_RI);
   }
 
   /**
@@ -844,7 +834,7 @@ public class SplitTableRegionProcedure
   private void postSplitRegion(final MasterProcedureEnv env) throws IOException {
     final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
     if (cpHost != null) {
-      cpHost.postCompletedSplitRegionAction(daughterOneRI, daughterTwoRI, getUser());
+      cpHost.postCompletedSplitRegionAction(daughter_1_RI, daughter_2_RI, getUser());
     }
   }
 
@@ -862,8 +852,8 @@ public class SplitTableRegionProcedure
   private TransitRegionStateProcedure[] createAssignProcedures(MasterProcedureEnv env)
       throws IOException {
     List<RegionInfo> hris = new ArrayList<RegionInfo>(2);
-    hris.add(daughterOneRI);
-    hris.add(daughterTwoRI);
+    hris.add(daughter_1_RI);
+    hris.add(daughter_2_RI);
     return AssignmentManagerUtil.createAssignProceduresForOpeningNewRegions(env, hris,
       getRegionReplication(env), getParentRegionServerName(env));
   }
@@ -878,9 +868,9 @@ public class SplitTableRegionProcedure
     long maxSequenceId =
       WALSplitter.getMaxRegionSequenceId(walFS, getWALRegionDir(env, getParentRegion()));
     if (maxSequenceId > 0) {
-      WALSplitter.writeRegionSequenceIdFile(walFS, getWALRegionDir(env, daughterOneRI),
+      WALSplitter.writeRegionSequenceIdFile(walFS, getWALRegionDir(env, daughter_1_RI),
           maxSequenceId);
-      WALSplitter.writeRegionSequenceIdFile(walFS, getWALRegionDir(env, daughterTwoRI),
+      WALSplitter.writeRegionSequenceIdFile(walFS, getWALRegionDir(env, daughter_2_RI),
           maxSequenceId);
     }
   }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
index 4d19bd5..9666aa5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
@@ -807,12 +807,6 @@ public class HRegionFileSystem {
     Path mergedRegionTmpDir = this.getMergesDir(mergedRegionInfo);
     // Move the tmp dir in the expected location
     if (mergedRegionTmpDir != null && fs.exists(mergedRegionTmpDir)) {
-
-      // Write HRI to a file in case we need to recover hbase:meta
-      Path regionInfoFile = new Path(mergedRegionTmpDir, REGION_INFO_FILE);
-      byte[] regionInfoContent = getRegionInfoFileContent(regionInfo);
-      writeRegionInfoFileContent(conf, fs, regionInfoFile, regionInfoContent);
-
       if (!fs.rename(mergedRegionTmpDir, regionDir)) {
         throw new IOException("Unable to rename " + mergedRegionTmpDir + " to "
             + regionDir);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHbck.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHbck.java
index eb4a2fd..8318757 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHbck.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHbck.java
@@ -18,32 +18,17 @@
 package org.apache.hadoop.hbase.client;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
 import java.util.stream.Collectors;
-
-import org.apache.hadoop.hbase.Coprocessor;
-import org.apache.hadoop.hbase.CoprocessorEnvironment;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.coprocessor.MasterCoprocessor;
-import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
-import org.apache.hadoop.hbase.coprocessor.MasterObserver;
-import org.apache.hadoop.hbase.coprocessor.ObserverContext;
-import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.RegionState;
-import org.apache.hadoop.hbase.master.assignment.MergeTableRegionsProcedure;
-import org.apache.hadoop.hbase.master.assignment.SplitTableRegionProcedure;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.master.procedure.TableProcedureInterface;
 import org.apache.hadoop.hbase.procedure2.Procedure;
@@ -55,7 +40,6 @@ import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.AfterClass;
-import org.junit.Assert;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
@@ -73,7 +57,6 @@ import org.slf4j.LoggerFactory;
 import org.apache.hbase.thirdparty.com.google.common.io.Closeables;
 
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos;
 
 /**
  * Class to test HBaseHbck. Spins up the minicluster once at test start and then takes it down
@@ -119,12 +102,6 @@ public class TestHbck {
     TEST_UTIL.createMultiRegionTable(TABLE_NAME, Bytes.toBytes("family1"), 5);
     procExec = TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor();
     ASYNC_CONN = ConnectionFactory.createAsyncConnection(TEST_UTIL.getConfiguration()).get();
-    TEST_UTIL.getHBaseCluster().getMaster().getMasterCoprocessorHost().load(
-      FailingMergeAfterMetaUpdatedMasterObserver.class, Coprocessor.PRIORITY_USER,
-      TEST_UTIL.getHBaseCluster().getMaster().getConfiguration());
-    TEST_UTIL.getHBaseCluster().getMaster().getMasterCoprocessorHost().load(
-      FailingSplitAfterMetaUpdatedMasterObserver.class, Coprocessor.PRIORITY_USER,
-      TEST_UTIL.getHBaseCluster().getMaster().getConfiguration());
   }
 
   @AfterClass
@@ -228,101 +205,6 @@ public class TestHbck {
   }
 
   @Test
-  public void testRecoverMergeAfterMetaUpdated() throws Exception {
-    String testTable = async ? "mergeTestAsync" : "mergeTestSync";
-    TEST_UTIL.createMultiRegionTable(TableName.valueOf(testTable), Bytes.toBytes("family1"), 5);
-    TEST_UTIL.loadTable(TEST_UTIL.getConnection().getTable(TableName.valueOf(testTable)),
-      Bytes.toBytes("family1"), true);
-    HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
-    Hbck hbck = getHbck();
-    FailingMergeAfterMetaUpdatedMasterObserver observer = master.getMasterCoprocessorHost()
-        .findCoprocessor(FailingMergeAfterMetaUpdatedMasterObserver.class);
-    try (Admin admin = TEST_UTIL.getConnection().getAdmin()) {
-      List<RegionInfo> regions = admin.getRegions(TableName.valueOf(testTable));
-      admin.mergeRegionsAsync(regions.get(0).getRegionName(), regions.get(1).getRegionName(), true);
-      assertNotNull(observer);
-      observer.latch.await(5000, TimeUnit.MILLISECONDS);
-      Map<String, MasterProtos.REGION_ERROR_TYPE> result =
-          hbck.getFailedSplitMergeLegacyRegions(Arrays.asList(TableName.valueOf(testTable)));
-      Assert.assertEquals(0, result.size());
-      Optional<Procedure<?>> procedure = TEST_UTIL.getHBaseCluster().getMaster().getProcedures()
-          .stream().filter(p -> p instanceof MergeTableRegionsProcedure).findAny();
-      Assert.assertTrue(procedure.isPresent());
-      hbck.bypassProcedure(Arrays.asList(procedure.get().getProcId()), 5, true, false);
-      result = hbck.getFailedSplitMergeLegacyRegions(Arrays.asList(TableName.valueOf(testTable)));
-      Assert.assertEquals(1, result.size());
-      hbck.assigns(Arrays.asList(result.keySet().toArray(new String[0])).stream()
-          .map(regionName -> regionName.split("\\.")[1]).collect(Collectors.toList()));
-      ProcedureTestingUtility.waitNoProcedureRunning(master.getMasterProcedureExecutor());
-      // now the state should be fixed
-      result = hbck.getFailedSplitMergeLegacyRegions(Arrays.asList(TableName.valueOf(testTable)));
-      Assert.assertEquals(0, result.size());
-    } catch (InterruptedException ie) {
-      throw new IOException(ie);
-    } finally {
-      observer.resetLatch();
-    }
-  }
-
-  @Test
-  public void testRecoverSplitAfterMetaUpdated() throws Exception {
-    String testTable = async ? "splitTestAsync" : "splitTestSync";
-    TEST_UTIL.createMultiRegionTable(TableName.valueOf(testTable), Bytes.toBytes("family1"), 5);
-    HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
-    Hbck hbck = getHbck();
-    FailingSplitAfterMetaUpdatedMasterObserver observer = master.getMasterCoprocessorHost()
-        .findCoprocessor(FailingSplitAfterMetaUpdatedMasterObserver.class);
-    assertNotNull(observer);
-    try (Admin admin = TEST_UTIL.getConnection().getAdmin()) {
-      byte[] splitKey = Bytes.toBytes("bcd");
-      admin.split(TableName.valueOf(testTable), splitKey);
-      observer.latch.await(5000, TimeUnit.MILLISECONDS);
-      Map<String, MasterProtos.REGION_ERROR_TYPE> result =
-          hbck.getFailedSplitMergeLegacyRegions(Arrays.asList(TableName.valueOf(testTable)));
-      // since there is a split procedure work on the region, thus this check should return a empty
-      // map.
-      Assert.assertEquals(0, result.size());
-      Optional<Procedure<?>> procedure = TEST_UTIL.getHBaseCluster().getMaster().getProcedures()
-          .stream().filter(p -> p instanceof SplitTableRegionProcedure).findAny();
-      Assert.assertTrue(procedure.isPresent());
-      hbck.bypassProcedure(Arrays.asList(procedure.get().getProcId()), 5, true, false);
-      result = hbck.getFailedSplitMergeLegacyRegions(Arrays.asList(TableName.valueOf(testTable)));
-      Assert.assertEquals(2, result.size());
-      hbck.assigns(Arrays.asList(result.keySet().toArray(new String[0])).stream()
-          .map(regionName -> regionName.split("\\.")[1]).collect(Collectors.toList()));
-      ProcedureTestingUtility.waitNoProcedureRunning(master.getMasterProcedureExecutor());
-      // now the state should be fixed
-      result = hbck.getFailedSplitMergeLegacyRegions(Arrays.asList(TableName.valueOf(testTable)));
-      Assert.assertEquals(0, result.size());
-
-      //split one of the daughter region again
-      observer.resetLatch();
-      byte[] splitKey2 = Bytes.toBytes("bcde");
-
-      admin.split(TableName.valueOf(testTable), splitKey2);
-      observer.latch.await(5000, TimeUnit.MILLISECONDS);
-
-      procedure = TEST_UTIL.getHBaseCluster().getMaster().getProcedures()
-          .stream().filter(p -> p instanceof SplitTableRegionProcedure).findAny();
-      Assert.assertTrue(procedure.isPresent());
-      hbck.bypassProcedure(Arrays.asList(procedure.get().getProcId()), 5, true, false);
-      result = hbck.getFailedSplitMergeLegacyRegions(Arrays.asList(TableName.valueOf(testTable)));
-      Assert.assertEquals(2, result.size());
-      hbck.assigns(Arrays.asList(result.keySet().toArray(new String[0])).stream()
-          .map(regionName -> regionName.split("\\.")[1]).collect(Collectors.toList()));
-      ProcedureTestingUtility.waitNoProcedureRunning(master.getMasterProcedureExecutor());
-      // now the state should be fixed
-      result = hbck.getFailedSplitMergeLegacyRegions(Arrays.asList(TableName.valueOf(testTable)));
-      Assert.assertEquals(0, result.size());
-    } catch (InterruptedException ie) {
-      throw new IOException(ie);
-    } finally {
-      observer.resetLatch();
-    }
-  }
-
-
-  @Test
   public void testScheduleSCP() throws Exception {
     HRegionServer testRs = TEST_UTIL.getRSForFirstRegionInTable(TABLE_NAME);
     TEST_UTIL.loadTable(TEST_UTIL.getConnection().getTable(TABLE_NAME), Bytes.toBytes("family1"),
@@ -341,60 +223,6 @@ public class TestHbck {
     waitOnPids(pids);
   }
 
-  public static class FailingSplitAfterMetaUpdatedMasterObserver
-      implements MasterCoprocessor, MasterObserver {
-    public volatile CountDownLatch latch;
-
-    @Override
-    public void start(CoprocessorEnvironment e) throws IOException {
-      resetLatch();
-    }
-
-    @Override
-    public Optional<MasterObserver> getMasterObserver() {
-      return Optional.of(this);
-    }
-
-    @Override
-    public void preSplitRegionAfterMETAAction(ObserverContext<MasterCoprocessorEnvironment> ctx)
-        throws IOException {
-      LOG.info("I'm here");
-      latch.countDown();
-      throw new IOException("this procedure will fail at here forever");
-    }
-
-    public void resetLatch() {
-      this.latch = new CountDownLatch(1);
-    }
-  }
-
-  public static class FailingMergeAfterMetaUpdatedMasterObserver
-      implements MasterCoprocessor, MasterObserver {
-    public volatile CountDownLatch latch;
-
-    @Override
-    public void start(CoprocessorEnvironment e) throws IOException {
-      resetLatch();
-    }
-
-    @Override
-    public Optional<MasterObserver> getMasterObserver() {
-      return Optional.of(this);
-    }
-
-    public void resetLatch() {
-      this.latch = new CountDownLatch(1);
-    }
-
-    @Override
-    public void postMergeRegionsCommitAction(
-        final ObserverContext<MasterCoprocessorEnvironment> ctx, final RegionInfo[] regionsToMerge,
-        final RegionInfo mergedRegion) throws IOException {
-      latch.countDown();
-      throw new IOException("this procedure will fail at here forever");
-    }
-  }
-
   private void waitOnPids(List<Long> pids) {
     TEST_UTIL.waitFor(60000, () -> pids.stream().allMatch(procExec::isFinished));
   }


[hbase] 10/49: HBASE-21926 Profiler servlet

Posted by op...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

openinx pushed a commit to branch HBASE-21879
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit 2e3f1f0b82686374c2d59553e77f9562ff06dec4
Author: Andrew Purtell <ap...@apache.org>
AuthorDate: Sun Mar 17 18:47:52 2019 -0700

    HBASE-21926 Profiler servlet
---
 bin/hbase                                          |   1 +
 .../org/apache/hadoop/hbase/http/HttpServer.java   |  24 +-
 .../hadoop/hbase/http/ProfileOutputServlet.java    |  58 ++++
 .../apache/hadoop/hbase/http/ProfileServlet.java   | 373 +++++++++++++++++++++
 .../org/apache/hadoop/hbase/util/ProcessUtils.java |  70 ++++
 .../src/main/resources/hbase-webapps/rest/rest.jsp |   1 +
 .../hbase/tmpl/master/MasterStatusTmpl.jamon       |   1 +
 .../hbase/tmpl/regionserver/RSStatusTmpl.jamon     |   1 +
 .../main/resources/hbase-webapps/master/header.jsp |   1 +
 .../hbase-webapps/regionserver/header.jsp          |   1 +
 .../main/resources/hbase-webapps/thrift/thrift.jsp |   1 +
 src/main/asciidoc/_chapters/profiler.adoc          |  98 ++++++
 src/main/asciidoc/book.adoc                        |   1 +
 13 files changed, 627 insertions(+), 4 deletions(-)

diff --git a/bin/hbase b/bin/hbase
index 59a3b69..1af2213 100755
--- a/bin/hbase
+++ b/bin/hbase
@@ -689,5 +689,6 @@ fi
 if [ "${HBASE_NOEXEC}" != "" ]; then
   "$JAVA" -Dproc_$COMMAND -XX:OnOutOfMemoryError="kill -9 %p" $HEAP_SETTINGS $HBASE_OPTS $CLASS "$@"
 else
+  export JVM_PID="$$"
   exec "$JAVA" -Dproc_$COMMAND -XX:OnOutOfMemoryError="kill -9 %p" $HEAP_SETTINGS $HBASE_OPTS $CLASS "$@"
 fi
diff --git a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/HttpServer.java b/hbase-http/src/main/java/org/apache/hadoop/hbase/http/HttpServer.java
index af72ab8..e9bcfbb 100644
--- a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/HttpServer.java
+++ b/hbase-http/src/main/java/org/apache/hadoop/hbase/http/HttpServer.java
@@ -26,6 +26,9 @@ import java.net.InetSocketAddress;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URL;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Enumeration;
@@ -570,7 +573,7 @@ public class HttpServer implements FilterContainer {
       }
     }
 
-    addDefaultServlets();
+    addDefaultServlets(contexts);
 
     if (pathSpecs != null) {
       for (String path : pathSpecs) {
@@ -668,11 +671,10 @@ public class HttpServer implements FilterContainer {
   /**
    * Add default servlets.
    */
-  protected void addDefaultServlets() {
+  protected void addDefaultServlets(ContextHandlerCollection contexts) throws IOException {
     // set up default servlets
     addServlet("stacks", "/stacks", StackServlet.class);
     addServlet("logLevel", "/logLevel", LogLevel.Servlet.class);
-
     // Hadoop3 has moved completely to metrics2, and  dropped support for Metrics v1's
     // MetricsServlet (see HADOOP-12504).  We'll using reflection to load if against hadoop2.
     // Remove when we drop support for hbase on hadoop2.x.
@@ -682,9 +684,23 @@ public class HttpServer implements FilterContainer {
     } catch (Exception e) {
       // do nothing
     }
-
     addServlet("jmx", "/jmx", JMXJsonServlet.class);
     addServlet("conf", "/conf", ConfServlet.class);
+    final String asyncProfilerHome = ProfileServlet.getAsyncProfilerHome();
+    if (asyncProfilerHome != null && !asyncProfilerHome.trim().isEmpty()) {
+      addServlet("prof", "/prof", ProfileServlet.class);
+      Path tmpDir = Paths.get(ProfileServlet.OUTPUT_DIR);
+      if (Files.notExists(tmpDir)) {
+        Files.createDirectories(tmpDir);
+      }
+      ServletContextHandler genCtx = new ServletContextHandler(contexts, "/prof-output");
+      genCtx.addServlet(ProfileOutputServlet.class, "/*");
+      genCtx.setResourceBase(tmpDir.toAbsolutePath().toString());
+      genCtx.setDisplayName("prof-output");
+    } else {
+      LOG.info("ASYNC_PROFILER_HOME environment variable and async.profiler.home system property " +
+        "not specified. Disabling /prof endpoint.");
+    }
   }
 
   /**
diff --git a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/ProfileOutputServlet.java b/hbase-http/src/main/java/org/apache/hadoop/hbase/http/ProfileOutputServlet.java
new file mode 100644
index 0000000..670c3ac
--- /dev/null
+++ b/hbase-http/src/main/java/org/apache/hadoop/hbase/http/ProfileOutputServlet.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.http;
+
+import java.io.File;
+import java.io.IOException;
+
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import org.apache.yetus.audience.InterfaceAudience;
+import org.eclipse.jetty.servlet.DefaultServlet;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Servlet to serve files generated by {@link ProfileServlet}
+ */
+@InterfaceAudience.Private
+public class ProfileOutputServlet extends DefaultServlet {
+  private static final long serialVersionUID = 1L;
+  private static final Logger LOG = LoggerFactory.getLogger(ProfileOutputServlet.class);
+  private static final int REFRESH_PERIOD = 2;
+
+  @Override
+  protected void doGet(final HttpServletRequest req, final HttpServletResponse resp)
+    throws ServletException, IOException {
+    String absoluteDiskPath = getServletContext().getRealPath(req.getPathInfo());
+    File requestedFile = new File(absoluteDiskPath);
+    // async-profiler version 1.4 writes 'Started [cpu] profiling' to output file when profiler is
+    // running which gets replaced by final output. If final output is not ready yet, the file size
+    // will be <100 bytes (in all modes).
+    if (requestedFile.length() < 100) {
+      LOG.info(requestedFile  + " is incomplete. Sending auto-refresh header.");
+      resp.setHeader("Refresh", REFRESH_PERIOD + "," + req.getRequestURI());
+      resp.getWriter().write("This page will be auto-refreshed every " + REFRESH_PERIOD +
+        " seconds until the output file is ready.");
+    } else {
+      super.doGet(req, resp);
+    }
+  }
+}
diff --git a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/ProfileServlet.java b/hbase-http/src/main/java/org/apache/hadoop/hbase/http/ProfileServlet.java
new file mode 100644
index 0000000..25255a0
--- /dev/null
+++ b/hbase-http/src/main/java/org/apache/hadoop/hbase/http/ProfileServlet.java
@@ -0,0 +1,373 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.http;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import org.apache.hadoop.hbase.util.ProcessUtils;
+import org.apache.hbase.thirdparty.com.google.common.base.Joiner;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Servlet that runs async-profiler as web-endpoint.
+ * Following options from async-profiler can be specified as query paramater.
+ * //  -e event          profiling event: cpu|alloc|lock|cache-misses etc.
+ * //  -d duration       run profiling for 'duration' seconds (integer)
+ * //  -i interval       sampling interval in nanoseconds (long)
+ * //  -j jstackdepth    maximum Java stack depth (integer)
+ * //  -b bufsize        frame buffer size (long)
+ * //  -t                profile different threads separately
+ * //  -s                simple class names instead of FQN
+ * //  -o fmt[,fmt...]   output format: summary|traces|flat|collapsed|svg|tree|jfr
+ * //  --width px        SVG width pixels (integer)
+ * //  --height px       SVG frame height pixels (integer)
+ * //  --minwidth px     skip frames smaller than px (double)
+ * //  --reverse         generate stack-reversed FlameGraph / Call tree
+ * Example:
+ * - To collect 30 second CPU profile of current process (returns FlameGraph svg)
+ * curl "http://localhost:10002/prof"
+ * - To collect 1 minute CPU profile of current process and output in tree format (html)
+ * curl "http://localhost:10002/prof?output=tree&amp;duration=60"
+ * - To collect 30 second heap allocation profile of current process (returns FlameGraph svg)
+ * curl "http://localhost:10002/prof?event=alloc"
+ * - To collect lock contention profile of current process (returns FlameGraph svg)
+ * curl "http://localhost:10002/prof?event=lock"
+ * Following event types are supported (default is 'cpu') (NOTE: not all OS'es support all events)
+ * // Perf events:
+ * //    cpu
+ * //    page-faults
+ * //    context-switches
+ * //    cycles
+ * //    instructions
+ * //    cache-references
+ * //    cache-misses
+ * //    branches
+ * //    branch-misses
+ * //    bus-cycles
+ * //    L1-dcache-load-misses
+ * //    LLC-load-misses
+ * //    dTLB-load-misses
+ * //    mem:breakpoint
+ * //    trace:tracepoint
+ * // Java events:
+ * //    alloc
+ * //    lock
+ */
+@InterfaceAudience.Private
+public class ProfileServlet extends HttpServlet {
+  private static final long serialVersionUID = 1L;
+  private static final Logger LOG = LoggerFactory.getLogger(ProfileServlet.class);
+
+  private static final String ACCESS_CONTROL_ALLOW_METHODS = "Access-Control-Allow-Methods";
+  private static final String ALLOWED_METHODS = "GET";
+  private static final String ACCESS_CONTROL_ALLOW_ORIGIN = "Access-Control-Allow-Origin";
+  private static final String CONTENT_TYPE_TEXT = "text/plain; charset=utf-8";
+  private static final String ASYNC_PROFILER_HOME_ENV = "ASYNC_PROFILER_HOME";
+  private static final String ASYNC_PROFILER_HOME_SYSTEM_PROPERTY = "async.profiler.home";
+  private static final String PROFILER_SCRIPT = "/profiler.sh";
+  private static final int DEFAULT_DURATION_SECONDS = 10;
+  private static final AtomicInteger ID_GEN = new AtomicInteger(0);
+  static final String OUTPUT_DIR = System.getProperty("java.io.tmpdir") + "/prof-output";
+
+  enum Event {
+    CPU("cpu"),
+    ALLOC("alloc"),
+    LOCK("lock"),
+    PAGE_FAULTS("page-faults"),
+    CONTEXT_SWITCHES("context-switches"),
+    CYCLES("cycles"),
+    INSTRUCTIONS("instructions"),
+    CACHE_REFERENCES("cache-references"),
+    CACHE_MISSES("cache-misses"),
+    BRANCHES("branches"),
+    BRANCH_MISSES("branch-misses"),
+    BUS_CYCLES("bus-cycles"),
+    L1_DCACHE_LOAD_MISSES("L1-dcache-load-misses"),
+    LLC_LOAD_MISSES("LLC-load-misses"),
+    DTLB_LOAD_MISSES("dTLB-load-misses"),
+    MEM_BREAKPOINT("mem:breakpoint"),
+    TRACE_TRACEPOINT("trace:tracepoint"),;
+
+    private final String internalName;
+
+    Event(final String internalName) {
+      this.internalName = internalName;
+    }
+
+    public String getInternalName() {
+      return internalName;
+    }
+
+    public static Event fromInternalName(final String name) {
+      for (Event event : values()) {
+        if (event.getInternalName().equalsIgnoreCase(name)) {
+          return event;
+        }
+      }
+
+      return null;
+    }
+  }
+
+  enum Output {
+    SUMMARY,
+    TRACES,
+    FLAT,
+    COLLAPSED,
+    SVG,
+    TREE,
+    JFR
+  }
+
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "SE_TRANSIENT_FIELD_NOT_RESTORED",
+    justification = "This class is never serialized nor restored.")
+  private transient Lock profilerLock = new ReentrantLock();
+  private transient volatile Process process;
+  private String asyncProfilerHome;
+  private Integer pid;
+
+  public ProfileServlet() {
+    this.asyncProfilerHome = getAsyncProfilerHome();
+    this.pid = ProcessUtils.getPid();
+    LOG.info("Servlet process PID: " + pid + " asyncProfilerHome: " + asyncProfilerHome);
+  }
+
+  @Override
+  protected void doGet(final HttpServletRequest req, final HttpServletResponse resp)
+      throws IOException {
+    if (!HttpServer.isInstrumentationAccessAllowed(getServletContext(), req, resp)) {
+      resp.setStatus(HttpServletResponse.SC_UNAUTHORIZED);
+      setResponseHeader(resp);
+      resp.getWriter().write("Unauthorized: Instrumentation access is not allowed!");
+      return;
+    }
+
+    // make sure async profiler home is set
+    if (asyncProfilerHome == null || asyncProfilerHome.trim().isEmpty()) {
+      resp.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
+      setResponseHeader(resp);
+      resp.getWriter().write("ASYNC_PROFILER_HOME env is not set.");
+      return;
+    }
+
+    // if pid is explicitly specified, use it else default to current process
+    pid = getInteger(req, "pid", pid);
+
+    // if pid is not specified in query param and if current process pid cannot be determined
+    if (pid == null) {
+      resp.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
+      setResponseHeader(resp);
+      resp.getWriter().write(
+        "'pid' query parameter unspecified or unable to determine PID of current process.");
+      return;
+    }
+
+    final int duration = getInteger(req, "duration", DEFAULT_DURATION_SECONDS);
+    final Output output = getOutput(req);
+    final Event event = getEvent(req);
+    final Long interval = getLong(req, "interval");
+    final Integer jstackDepth = getInteger(req, "jstackdepth", null);
+    final Long bufsize = getLong(req, "bufsize");
+    final boolean thread = req.getParameterMap().containsKey("thread");
+    final boolean simple = req.getParameterMap().containsKey("simple");
+    final Integer width = getInteger(req, "width", null);
+    final Integer height = getInteger(req, "height", null);
+    final Double minwidth = getMinWidth(req);
+    final boolean reverse = req.getParameterMap().containsKey("reverse");
+
+    if (process == null || !process.isAlive()) {
+      try {
+        int lockTimeoutSecs = 3;
+        if (profilerLock.tryLock(lockTimeoutSecs, TimeUnit.SECONDS)) {
+          try {
+            File outputFile = new File(OUTPUT_DIR, "async-prof-pid-" + pid + "-" +
+              event.name().toLowerCase() + "-" + ID_GEN.incrementAndGet() + "." +
+              output.name().toLowerCase());
+            List<String> cmd = new ArrayList<>();
+            cmd.add(asyncProfilerHome + PROFILER_SCRIPT);
+            cmd.add("-e");
+            cmd.add(event.getInternalName());
+            cmd.add("-d");
+            cmd.add("" + duration);
+            cmd.add("-o");
+            cmd.add(output.name().toLowerCase());
+            cmd.add("-f");
+            cmd.add(outputFile.getAbsolutePath());
+            if (interval != null) {
+              cmd.add("-i");
+              cmd.add(interval.toString());
+            }
+            if (jstackDepth != null) {
+              cmd.add("-j");
+              cmd.add(jstackDepth.toString());
+            }
+            if (bufsize != null) {
+              cmd.add("-b");
+              cmd.add(bufsize.toString());
+            }
+            if (thread) {
+              cmd.add("-t");
+            }
+            if (simple) {
+              cmd.add("-s");
+            }
+            if (width != null) {
+              cmd.add("--width");
+              cmd.add(width.toString());
+            }
+            if (height != null) {
+              cmd.add("--height");
+              cmd.add(height.toString());
+            }
+            if (minwidth != null) {
+              cmd.add("--minwidth");
+              cmd.add(minwidth.toString());
+            }
+            if (reverse) {
+              cmd.add("--reverse");
+            }
+            cmd.add(pid.toString());
+            process = ProcessUtils.runCmdAsync(cmd);
+
+            // set response and set refresh header to output location
+            setResponseHeader(resp);
+            resp.setStatus(HttpServletResponse.SC_ACCEPTED);
+            String relativeUrl = "/prof-output/" + outputFile.getName();
+            resp.getWriter().write(
+              "Started [" + event.getInternalName() +
+              "] profiling. This page will automatically redirect to " +
+              relativeUrl + " after " + duration + " seconds.\n\ncommand:\n" +
+              Joiner.on(" ").join(cmd));
+
+            // to avoid auto-refresh by ProfileOutputServlet, refreshDelay can be specified
+            // via url param
+            int refreshDelay = getInteger(req, "refreshDelay", 0);
+
+            // instead of sending redirect, set auto-refresh so that browsers will refresh
+            // with redirected url
+            resp.setHeader("Refresh", (duration + refreshDelay) + ";" + relativeUrl);
+            resp.getWriter().flush();
+          } finally {
+            profilerLock.unlock();
+          }
+        } else {
+          setResponseHeader(resp);
+          resp.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
+          resp.getWriter().write(
+            "Unable to acquire lock. Another instance of profiler might be running.");
+          LOG.warn("Unable to acquire lock in " + lockTimeoutSecs +
+            " seconds. Another instance of profiler might be running.");
+        }
+      } catch (InterruptedException e) {
+        LOG.warn("Interrupted while acquiring profile lock.", e);
+        resp.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
+      }
+    } else {
+      setResponseHeader(resp);
+      resp.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
+      resp.getWriter().write("Another instance of profiler is already running.");
+    }
+  }
+
+  private Integer getInteger(final HttpServletRequest req, final String param,
+      final Integer defaultValue) {
+    final String value = req.getParameter(param);
+    if (value != null) {
+      try {
+        return Integer.valueOf(value);
+      } catch (NumberFormatException e) {
+        return defaultValue;
+      }
+    }
+    return defaultValue;
+  }
+
+  private Long getLong(final HttpServletRequest req, final String param) {
+    final String value = req.getParameter(param);
+    if (value != null) {
+      try {
+        return Long.valueOf(value);
+      } catch (NumberFormatException e) {
+        return null;
+      }
+    }
+    return null;
+  }
+
+  private Double getMinWidth(final HttpServletRequest req) {
+    final String value = req.getParameter("minwidth");
+    if (value != null) {
+      try {
+        return Double.valueOf(value);
+      } catch (NumberFormatException e) {
+        return null;
+      }
+    }
+    return null;
+  }
+
+  private Event getEvent(final HttpServletRequest req) {
+    final String eventArg = req.getParameter("event");
+    if (eventArg != null) {
+      Event event = Event.fromInternalName(eventArg);
+      return event == null ? Event.CPU : event;
+    }
+    return Event.CPU;
+  }
+
+  private Output getOutput(final HttpServletRequest req) {
+    final String outputArg = req.getParameter("output");
+    if (req.getParameter("output") != null) {
+      try {
+        return Output.valueOf(outputArg.trim().toUpperCase());
+      } catch (IllegalArgumentException e) {
+        return Output.SVG;
+      }
+    }
+    return Output.SVG;
+  }
+
+  private void setResponseHeader(final HttpServletResponse response) {
+    response.setHeader(ACCESS_CONTROL_ALLOW_METHODS, ALLOWED_METHODS);
+    response.setHeader(ACCESS_CONTROL_ALLOW_ORIGIN, "*");
+    response.setContentType(CONTENT_TYPE_TEXT);
+  }
+
+  static String getAsyncProfilerHome() {
+    String asyncProfilerHome = System.getenv(ASYNC_PROFILER_HOME_ENV);
+    // if ENV is not set, see if -Dasync.profiler.home=/path/to/async/profiler/home is set
+    if (asyncProfilerHome == null || asyncProfilerHome.trim().isEmpty()) {
+      asyncProfilerHome = System.getProperty(ASYNC_PROFILER_HOME_SYSTEM_PROPERTY);
+    }
+
+    return asyncProfilerHome;
+  }
+}
\ No newline at end of file
diff --git a/hbase-http/src/main/java/org/apache/hadoop/hbase/util/ProcessUtils.java b/hbase-http/src/main/java/org/apache/hadoop/hbase/util/ProcessUtils.java
new file mode 100644
index 0000000..2d3835e
--- /dev/null
+++ b/hbase-http/src/main/java/org/apache/hadoop/hbase/util/ProcessUtils.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.util;
+
+import java.io.IOException;
+import java.lang.management.ManagementFactory;
+import java.util.List;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Process related utilities.
+ */
+@InterfaceAudience.Private
+public final class ProcessUtils {
+  private static Logger LOG = LoggerFactory.getLogger(ProcessUtils.class);
+
+  private ProcessUtils() { }
+
+  public static Integer getPid() {
+    // JVM_PID is exported by bin/hbase run script
+    String pidStr = System.getenv("JVM_PID");
+
+    // in case if it is not set correctly used fallback from mxbean which is implementation specific
+    if (pidStr == null || pidStr.trim().isEmpty()) {
+      String name = ManagementFactory.getRuntimeMXBean().getName();
+      if (name != null) {
+        int idx = name.indexOf("@");
+        if (idx != -1) {
+          pidStr = name.substring(0, name.indexOf("@"));
+        }
+      }
+    }
+    try {
+      if (pidStr != null) {
+        return Integer.valueOf(pidStr);
+      }
+    } catch (NumberFormatException nfe) {
+      // ignore
+    }
+    return null;
+  }
+
+  public static Process runCmdAsync(List<String> cmd) {
+    try {
+      LOG.info("Running command async: " + cmd);
+      return new ProcessBuilder(cmd).inheritIO().start();
+    } catch (IOException ex) {
+      throw new IllegalStateException(ex);
+    }
+  }
+}
diff --git a/hbase-rest/src/main/resources/hbase-webapps/rest/rest.jsp b/hbase-rest/src/main/resources/hbase-webapps/rest/rest.jsp
index ba3c027..ed4e9c2 100644
--- a/hbase-rest/src/main/resources/hbase-webapps/rest/rest.jsp
+++ b/hbase-rest/src/main/resources/hbase-webapps/rest/rest.jsp
@@ -58,6 +58,7 @@ String listenPort = conf.get("hbase.rest.port", "8080");
                   <li><a href="/logs/">Local logs</a></li>
                   <li><a href="/logLevel">Log Level</a></li>
                   <li><a href="/jmx">Metrics Dump</a></li>
+                  <li><a href="/prof">Profiler</a></li>
                   <% if (HBaseConfiguration.isShowConfInServlet()) { %>
                   <li><a href="/conf">HBase Configuration</a></li>
                   <% } %>
diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon
index da44052..f6ea764 100644
--- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon
+++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon
@@ -156,6 +156,7 @@ AssignmentManager assignmentManager = master.getAssignmentManager();
                 <li><a href="/logLevel">Log Level</a></li>
                 <li><a href="/dump">Debug Dump</a></li>
                 <li><a href="/jmx">Metrics Dump</a></li>
+                <li><a href="/prof">Profiler</a></li>
                 <%if HBaseConfiguration.isShowConfInServlet()%>
                 <li><a href="/conf">HBase Configuration</a></li>
                 </%if>
diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RSStatusTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RSStatusTmpl.jamon
index 8eb362a..94fcecf 100644
--- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RSStatusTmpl.jamon
+++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RSStatusTmpl.jamon
@@ -113,6 +113,7 @@ org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
                 <li><a href="/logLevel">Log Level</a></li>
                 <li><a href="/dump">Debug Dump</a></li>
                 <li><a href="/jmx">Metrics Dump</a></li>
+                <li><a href="/prof">Profiler</a></li>
                 <%if HBaseConfiguration.isShowConfInServlet()%>
                 <li><a href="/conf">HBase Configuration</a></li>
                 </%if>
diff --git a/hbase-server/src/main/resources/hbase-webapps/master/header.jsp b/hbase-server/src/main/resources/hbase-webapps/master/header.jsp
index 0df7099..1e02e2b 100644
--- a/hbase-server/src/main/resources/hbase-webapps/master/header.jsp
+++ b/hbase-server/src/main/resources/hbase-webapps/master/header.jsp
@@ -64,6 +64,7 @@
             <li><a href="/logLevel">Log Level</a></li>
             <li><a href="/dump">Debug Dump</a></li>
             <li><a href="/jmx">Metrics Dump</a></li>
+            <li><a href="/prof">Profiler</a></li>
             <% if (HBaseConfiguration.isShowConfInServlet()) { %>
             <li><a href="/conf">HBase Configuration</a></li>
             <% } %>
diff --git a/hbase-server/src/main/resources/hbase-webapps/regionserver/header.jsp b/hbase-server/src/main/resources/hbase-webapps/regionserver/header.jsp
index d61cf10..88ca515 100644
--- a/hbase-server/src/main/resources/hbase-webapps/regionserver/header.jsp
+++ b/hbase-server/src/main/resources/hbase-webapps/regionserver/header.jsp
@@ -56,6 +56,7 @@
             <li><a href="/logLevel">Log Level</a></li>
             <li><a href="/dump">Debug Dump</a></li>
             <li><a href="/jmx">Metrics Dump</a></li>
+            <li><a href="/prof">Profiler</a></li>
             <% if (HBaseConfiguration.isShowConfInServlet()) { %>
             <li><a href="/conf">HBase Configuration</a></li>
             <% } %>
diff --git a/hbase-thrift/src/main/resources/hbase-webapps/thrift/thrift.jsp b/hbase-thrift/src/main/resources/hbase-webapps/thrift/thrift.jsp
index b0064b4..432b0f4 100644
--- a/hbase-thrift/src/main/resources/hbase-webapps/thrift/thrift.jsp
+++ b/hbase-thrift/src/main/resources/hbase-webapps/thrift/thrift.jsp
@@ -66,6 +66,7 @@ String compact = conf.get("hbase.regionserver.thrift.compact", "false");
                 <li><a href="/logs/">Local logs</a></li>
                 <li><a href="/logLevel">Log Level</a></li>
                 <li><a href="/jmx">Metrics Dump</a></li>
+                <li><a href="/prof">Profiler</a></li>
                 <% if (HBaseConfiguration.isShowConfInServlet()) { %>
                 <li><a href="/conf">HBase Configuration</a></li>
                 <% } %>
diff --git a/src/main/asciidoc/_chapters/profiler.adoc b/src/main/asciidoc/_chapters/profiler.adoc
new file mode 100644
index 0000000..09cbccc
--- /dev/null
+++ b/src/main/asciidoc/_chapters/profiler.adoc
@@ -0,0 +1,98 @@
+////
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+////
+
+[[profiler]]
+= Profiler Servlet
+:doctype: book
+:numbered:
+:toc: left
+:icons: font
+:experimental:
+
+== Background
+
+HBASE-21926 introduced a new servlet that supports integrated profiling via async-profiler.
+
+== Prerequisites
+
+Go to https://github.com/jvm-profiling-tools/async-profiler, download a release appropriate for your platform, and install on every cluster host.
+
+Set `ASYNC_PROFILER_HOME` in the environment (put it in hbase-env.sh) to the root directory of the async-profiler install location, or pass it on the HBase daemon's command line as a system property as `-Dasync.profiler.home=/path/to/async-profiler`.
+
+== Usage
+
+Once the prerequisites are satisfied, access to async-profiler is available by way of the HBase UI or direct interaction with the infoserver.
+
+Examples:
+
+* To collect 30 second CPU profile of current process (returns FlameGraph svg)
+  `curl http://localhost:16030/prof`
+* To collect 1 minute CPU profile of current process and output in tree format (html)
+  `curl http://localhost:16030/prof?output=tree&duration=60`
+* To collect 30 second heap allocation profile of current process (returns FlameGraph svg)
+  `curl http://localhost:16030/prof?event=alloc`
+* To collect lock contention profile of current process (returns FlameGraph svg)
+  `curl http://localhost:16030/prof?event=lock`
+
+The following event types are supported by async-profiler. Use the 'event' parameter to specify. Default is 'cpu'. Not all operating systems will support all types.
+
+Perf events:
+
+* cpu
+* page-faults
+* context-switches
+* cycles
+* instructions
+* cache-references
+* cache-misses
+* branches
+* branch-misses
+* bus-cycles
+* L1-dcache-load-misses
+* LLC-load-misses
+* dTLB-load-misses
+
+Java events:
+
+* alloc
+* lock
+
+The following output formats are supported. Use the 'output' parameter to specify. Default is 'flamegraph'.
+
+Output formats:
+
+* summary: A dump of basic profiling statistics.
+* traces: Call traces.
+* flat: Flat profile (top N hot methods).
+* collapsed: Collapsed call traces in the format used by FlameGraph script. This is a collection of call stacks, where each line is a semicolon separated list of frames followed by a counter.
+* svg: FlameGraph in SVG format.
+* tree: Call tree in HTML format.
+* jfr: Call traces in Java Flight Recorder format.
+
+The 'duration' parameter specifies how long to collect trace data before generating output, specified in seconds. The default is 10 seconds.
+
+== UI
+
+In the UI, there is a new entry 'Profiler' in the top menu that will run the default action, which is to profile the CPU usage of the local process for thirty seconds and then produce FlameGraph SVG output.
+
+== Notes
+
+The query parameter `pid` can be used to specify the process id of a specific process to be profiled. If this parameter is missing the local process in which the infoserver is embedded will be profiled. Profile targets that are not JVMs might work but is not specifically supported. There are security implications. Access to the infoserver should be appropriately restricted.
diff --git a/src/main/asciidoc/book.adoc b/src/main/asciidoc/book.adoc
index edfbcce..51a2800 100644
--- a/src/main/asciidoc/book.adoc
+++ b/src/main/asciidoc/book.adoc
@@ -73,6 +73,7 @@ include::_chapters/thrift_filter_language.adoc[]
 include::_chapters/spark.adoc[]
 include::_chapters/cp.adoc[]
 include::_chapters/performance.adoc[]
+include::_chapters/profiler.adoc[]
 include::_chapters/troubleshooting.adoc[]
 include::_chapters/case_studies.adoc[]
 include::_chapters/ops_mgt.adoc[]


[hbase] 34/49: Add 2.1.4 to the downloads page in place of 2.1.3

Posted by op...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

openinx pushed a commit to branch HBASE-21879
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit 8fc81c5930f5e9f12397bbbe34300c2b0efdeb7f
Author: stack <st...@apache.org>
AuthorDate: Tue Mar 26 10:26:06 2019 -0400

    Add 2.1.4 to the downloads page in place of 2.1.3
---
 src/site/xdoc/downloads.xml | 16 ++++++++--------
 1 file changed, 8 insertions(+), 8 deletions(-)

diff --git a/src/site/xdoc/downloads.xml b/src/site/xdoc/downloads.xml
index e99055e..c53e9f1 100644
--- a/src/site/xdoc/downloads.xml
+++ b/src/site/xdoc/downloads.xml
@@ -44,24 +44,24 @@ under the License.
     </tr>
     <tr>
       <td style="test-align: left">
-        2.1.3
+        2.1.4
       </td>
       <td style="test-align: left">
-        2019/02/14
+        2019/03/26
       </td>
       <td style="test-align: left">
-        <a href="https://apache.org/dist/hbase/2.1.3/compatibility_report_2.1.2vs2.1.3.html">2.1.2 vs 2.1.3</a>
+        <a href="https://apache.org/dist/hbase/2.1.4/api_compare_2.1.3_to_2.1.4RC1.html">2.1.3 vs 2.1.4</a>
       </td>
       <td style="test-align: left">
-        <a href="https://apache.org/dist/hbase/2.1.3/CHANGES.md">Changes</a>
+        <a href="https://apache.org/dist/hbase/2.1.4/CHANGES.md">Changes</a>
       </td>
       <td style="test-align: left">
-        <a href="https://apache.org/dist/hbase/2.1.3/RELEASENOTES.md">Release Notes</a>
+        <a href="https://apache.org/dist/hbase/2.1.4/RELEASENOTES.md">Release Notes</a>
       </td>
       <td style="test-align: left">
-        <a href="https://www.apache.org/dyn/closer.lua/hbase/2.1.3/hbase-2.1.3-src.tar.gz">src</a> (<a href="https://apache.org/dist/hbase/2.1.3/hbase-2.1.3-src.tar.gz.sha512">sha512</a> <a href="https://apache.org/dist/hbase/2.1.3/hbase-2.1.3-src.tar.gz.asc">asc</a>) <br />
-        <a href="https://www.apache.org/dyn/closer.lua/hbase/2.1.3/hbase-2.1.3-bin.tar.gz">bin</a> (<a href="https://apache.org/dist/hbase/2.1.3/hbase-2.1.3-bin.tar.gz.sha512">sha512</a> <a href="https://apache.org/dist/hbase/2.1.3/hbase-2.1.3-bin.tar.gz.asc">asc</a>) <br />
-        <a href="https://www.apache.org/dyn/closer.lua/hbase/2.1.3/hbase-2.1.3-client-bin.tar.gz">client-bin</a> (<a href="https://apache.org/dist/hbase/2.1.3/hbase-2.1.3-client-bin.tar.gz.sha512">sha512</a> <a href="https://apache.org/dist/hbase/2.1.3/hbase-2.1.3-client-bin.tar.gz.asc">asc</a>)
+        <a href="https://www.apache.org/dyn/closer.lua/hbase/2.1.4/hbase-2.1.4-src.tar.gz">src</a> (<a href="https://apache.org/dist/hbase/2.1.4/hbase-2.1.4-src.tar.gz.sha512">sha512</a> <a href="https://apache.org/dist/hbase/2.1.4/hbase-2.1.4-src.tar.gz.asc">asc</a>) <br />
+        <a href=4https://www.apache.org/dyn/closer.lua/hbase/2.1.4/hbase-2.1.4-bin.tar.gz">bin</a> (<a href="https://apache.org/dist/hbase/2.1.4/hbase-2.1.4-bin.tar.gz.sha512">sha512</a> <a href="https://apache.org/dist/hbase/2.1.4/hbase-2.1.4-bin.tar.gz.asc">asc</a>) <br />
+        <a href="https://www.apache.org/dyn/closer.lua/hbase/2.1.4/hbase-2.1.4-client-bin.tar.gz">client-bin</a> (<a href="https://apache.org/dist/hbase/2.1.4/hbase-2.1.4-client-bin.tar.gz.sha512">sha512</a> <a href="https://apache.org/dist/hbase/2.1.4/hbase-2.1.4-client-bin.tar.gz.asc">asc</a>)
       </td>
     </tr>
     <tr>


[hbase] 44/49: HBASE-22082 Should not use an instance to access static members, which will increases compilation costs.

Posted by op...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

openinx pushed a commit to branch HBASE-21879
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit ecf0826fc2818539f2180dd6b7e933847d8bc420
Author: 李小保 <li...@mininglamp.com>
AuthorDate: Fri Mar 22 08:36:52 2019 +0800

    HBASE-22082 Should not use an instance to access static members, which will increases compilation costs.
    
    Signed-off-by: stack <st...@apache.org>
---
 .../src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java     | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index 26ff24e..bfd026a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -3678,7 +3678,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       visitBatchOperations(true, miniBatchOp.getLastIndexExclusive(), (int index) -> {
         Mutation mutation = getMutation(index);
         if (mutation instanceof Put) {
-          region.updateCellTimestamps(familyCellMaps[index].values(), byteTS);
+          HRegion.updateCellTimestamps(familyCellMaps[index].values(), byteTS);
           miniBatchOp.incrementNumOfPuts();
         } else {
           region.prepareDeleteTimestamps(mutation, familyCellMaps[index], byteTS);


[hbase] 24/49: HBASE-22064 Remove Admin.deleteSnapshot(byte[])

Posted by op...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

openinx pushed a commit to branch HBASE-21879
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit d2006343a8c5fc8bd8659ac17ba4c0225e935006
Author: pingsutw <pi...@gmail.com>
AuthorDate: Sat Mar 23 21:27:12 2019 +0800

    HBASE-22064 Remove Admin.deleteSnapshot(byte[])
    
    Signed-off-by: zhangduo <zh...@apache.org>
---
 .../src/main/java/org/apache/hadoop/hbase/client/Admin.java    | 10 ----------
 .../main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java   |  5 -----
 .../org/apache/hadoop/hbase/client/TestSnapshotFromClient.java |  2 +-
 .../hadoop/hbase/client/TestSnapshotTemporaryDirectory.java    |  6 +++---
 .../hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java    |  2 +-
 .../hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java     |  2 +-
 .../org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java    |  5 -----
 7 files changed, 6 insertions(+), 26 deletions(-)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
index 82b259b..b27e93d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
@@ -1514,16 +1514,6 @@ public interface Admin extends Abortable, Closeable {
    *
    * @param snapshotName name of the snapshot
    * @throws IOException if a remote or network exception occurs
-   * @deprecated Since 2.2.0. Will be removed in 3.0.0. Use {@link #deleteSnapshot(String)} instead.
-   */
-  @Deprecated
-  void deleteSnapshot(byte[] snapshotName) throws IOException;
-
-  /**
-   * Delete an existing snapshot.
-   *
-   * @param snapshotName name of the snapshot
-   * @throws IOException if a remote or network exception occurs
    */
   void deleteSnapshot(String snapshotName) throws IOException;
 
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 51e126c..91f9584 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
@@ -2473,11 +2473,6 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public void deleteSnapshot(final byte[] snapshotName) throws IOException {
-    deleteSnapshot(Bytes.toString(snapshotName));
-  }
-
-  @Override
   public void deleteSnapshot(final String snapshotName) throws IOException {
     // make sure the snapshot is possibly valid
     TableName.isLegalFullyQualifiedTableName(Bytes.toBytes(snapshotName));
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java
index ad4b84c..45a3620 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java
@@ -216,7 +216,7 @@ public class TestSnapshotFromClient {
 
     // take a snapshot of the disabled table
     final String SNAPSHOT_NAME = "offlineTableSnapshot";
-    byte[] snapshot = Bytes.toBytes(SNAPSHOT_NAME);
+    String snapshot = SNAPSHOT_NAME;
 
     admin.snapshot(new SnapshotDescription(SNAPSHOT_NAME, TABLE_NAME,
         SnapshotType.DISABLED, null, -1, SnapshotManifestV1.DESCRIPTOR_VERSION));
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotTemporaryDirectory.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotTemporaryDirectory.java
index 589b31d..0f7cff1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotTemporaryDirectory.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotTemporaryDirectory.java
@@ -306,7 +306,7 @@ public class TestSnapshotTemporaryDirectory {
 
     // take a snapshot of the disabled table
     final String SNAPSHOT_NAME = "offlineTableSnapshot";
-    byte[] snapshot = Bytes.toBytes(SNAPSHOT_NAME);
+    String snapshot = SNAPSHOT_NAME;
     takeSnapshot(TABLE_NAME, SNAPSHOT_NAME, true);
     LOG.debug("Snapshot completed.");
 
@@ -390,8 +390,8 @@ public class TestSnapshotTemporaryDirectory {
             LOG);
 
     // take a snapshot of the disabled table
-    byte[] snapshot = Bytes.toBytes("testOfflineTableSnapshotWithEmptyRegion");
-    takeSnapshot(TABLE_NAME, Bytes.toString(snapshot), true);
+    String snapshot = "testOfflineTableSnapshotWithEmptyRegion";
+    takeSnapshot(TABLE_NAME, snapshot, true);
     LOG.debug("Snapshot completed.");
 
     // make sure we have the snapshot
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java
index fd183fc..20ab334 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java
@@ -313,7 +313,7 @@ public class TestSnapshotFromMaster {
 
     // take a snapshot of the table
     String snapshotName = "snapshot";
-    byte[] snapshotNameBytes = Bytes.toBytes(snapshotName);
+    String snapshotNameBytes = snapshotName;
     admin.snapshot(snapshotName, TABLE_NAME);
 
     LOG.info("After snapshot File-System state");
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java
index 2f84c95..e24d445 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java
@@ -186,7 +186,7 @@ public class TestFlushSnapshotFromClient {
 
     // take a snapshot of the enabled table
     String snapshotString = "skipFlushTableSnapshot";
-    byte[] snapshot = Bytes.toBytes(snapshotString);
+    String snapshot = snapshotString;
     admin.snapshot(snapshotString, TABLE_NAME, SnapshotType.SKIPFLUSH);
     LOG.debug("Snapshot completed.");
 
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java
index 508a623..85adf7d 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java
@@ -855,11 +855,6 @@ public class ThriftAdmin implements Admin {
   }
 
   @Override
-  public void deleteSnapshot(byte[] snapshotName) {
-    throw new NotImplementedException("deleteSnapshot not supported in ThriftAdmin");
-  }
-
-  @Override
   public void deleteSnapshot(String snapshotName) {
     throw new NotImplementedException("deleteSnapshot not supported in ThriftAdmin");
   }


[hbase] 19/49: HBASE-22063 Deprecated Admin.deleteSnapshot(byte[])

Posted by op...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

openinx pushed a commit to branch HBASE-21879
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit 4311e07b0c9e458895c77c16565aed6a4dbb1008
Author: xujunhong <xu...@xiaomi.com>
AuthorDate: Tue Mar 19 12:39:36 2019 +0000

    HBASE-22063 Deprecated Admin.deleteSnapshot(byte[])
    
    Signed-off-by: zhangduo <zh...@apache.org>
---
 hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java | 2 ++
 1 file changed, 2 insertions(+)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
index 4da5f6f..82b259b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
@@ -1514,7 +1514,9 @@ public interface Admin extends Abortable, Closeable {
    *
    * @param snapshotName name of the snapshot
    * @throws IOException if a remote or network exception occurs
+   * @deprecated Since 2.2.0. Will be removed in 3.0.0. Use {@link #deleteSnapshot(String)} instead.
    */
+  @Deprecated
   void deleteSnapshot(byte[] snapshotName) throws IOException;
 
   /**


[hbase] 49/49: HBASE-22100 False positive for error prone warnings in pre commit job

Posted by op...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

openinx pushed a commit to branch HBASE-21879
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit 109c3caa3f993aca42a82b36306bc33dd6356cf5
Author: zhangduo <zh...@apache.org>
AuthorDate: Wed Mar 27 17:03:02 2019 +0800

    HBASE-22100 False positive for error prone warnings in pre commit job
    
    Signed-off-by: Sean Busbey <bu...@apache.org>
---
 dev-support/hbase-personality.sh | 18 ++++++++++++++++++
 1 file changed, 18 insertions(+)

diff --git a/dev-support/hbase-personality.sh b/dev-support/hbase-personality.sh
index 8dd24be..90d1f77 100755
--- a/dev-support/hbase-personality.sh
+++ b/dev-support/hbase-personality.sh
@@ -706,6 +706,24 @@ function hbaseanti_patchfile
   return 0
 }
 
+## @description  process the javac output for generating WARNING/ERROR
+## @audience     private
+## @stability    evolving
+## @param        input filename
+## @param        output filename
+# Override the default javac_logfilter so that we can do a sort before outputing the WARNING/ERROR.
+# This is because that the output order of the error prone warnings is not stable, so the diff
+# method will report unexpected errors if we do not sort it. Notice that a simple sort will cause
+# line number being sorted by lexicographical so the output maybe a bit strange to human but it is
+# really hard to sort by file name first and then line number and column number in shell...
+function hbase_javac_logfilter
+{
+  declare input=$1
+  declare output=$2
+
+  ${GREP} -E '\[(ERROR|WARNING)\] /.*\.java:' "${input}" | sort > "${output}"
+}
+
 ## This is named so that yetus will check us right after running tests.
 ## Essentially, we check for normal failures and then we look for zombies.
 #function hbase_unit_logfilter


[hbase] 47/49: HBASE-21918 the use cases in doc should use Cell instead of KeyValue

Posted by op...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

openinx pushed a commit to branch HBASE-21879
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit 726d2d520e80f4b838fab38cad2324df00239e75
Author: Zheng Wang <18...@qq.com>
AuthorDate: Wed Feb 20 00:13:18 2019 +0800

    HBASE-21918 the use cases in doc should use Cell instead of KeyValue
    
    Signed-off-by: Xu Cang <xc...@salesforce.com>
    Signed-off-by: stack <st...@apache.org>
---
 src/main/asciidoc/_chapters/architecture.adoc | 12 ++++++------
 src/main/asciidoc/_chapters/datamodel.adoc    |  2 +-
 src/main/asciidoc/_chapters/mapreduce.adoc    |  4 ++--
 3 files changed, 9 insertions(+), 9 deletions(-)

diff --git a/src/main/asciidoc/_chapters/architecture.adoc b/src/main/asciidoc/_chapters/architecture.adoc
index 9a5eba9..9bdf179 100644
--- a/src/main/asciidoc/_chapters/architecture.adoc
+++ b/src/main/asciidoc/_chapters/architecture.adoc
@@ -452,8 +452,8 @@ scan.setFilter(f);
 scan.setBatch(10); // set this if there could be many columns returned
 ResultScanner rs = t.getScanner(scan);
 for (Result r = rs.next(); r != null; r = rs.next()) {
-  for (KeyValue kv : r.raw()) {
-    // each kv represents a column
+  for (Cell cell : result.listCells()) {
+    // each cell represents a column
   }
 }
 rs.close();
@@ -482,8 +482,8 @@ scan.setFilter(f);
 scan.setBatch(10); // set this if there could be many columns returned
 ResultScanner rs = t.getScanner(scan);
 for (Result r = rs.next(); r != null; r = rs.next()) {
-  for (KeyValue kv : r.raw()) {
-    // each kv represents a column
+  for (Cell cell : result.listCells()) {
+    // each cell represents a column
   }
 }
 rs.close();
@@ -518,8 +518,8 @@ scan.setFilter(f);
 scan.setBatch(10); // set this if there could be many columns returned
 ResultScanner rs = t.getScanner(scan);
 for (Result r = rs.next(); r != null; r = rs.next()) {
-  for (KeyValue kv : r.raw()) {
-    // each kv represents a column
+  for (Cell cell : result.listCells()) {
+    // each cell represents a column
   }
 }
 rs.close();
diff --git a/src/main/asciidoc/_chapters/datamodel.adoc b/src/main/asciidoc/_chapters/datamodel.adoc
index ba4961a..7d1aece 100644
--- a/src/main/asciidoc/_chapters/datamodel.adoc
+++ b/src/main/asciidoc/_chapters/datamodel.adoc
@@ -425,7 +425,7 @@ Get get = new Get(Bytes.toBytes("row1"));
 get.setMaxVersions(3);  // will return last 3 versions of row
 Result r = table.get(get);
 byte[] b = r.getValue(CF, ATTR);  // returns current version of value
-List<KeyValue> kv = r.getColumn(CF, ATTR);  // returns all versions of this column
+List<Cell> cells = r.getColumnCells(CF, ATTR);  // returns all versions of this column
 ----
 
 ==== Put
diff --git a/src/main/asciidoc/_chapters/mapreduce.adoc b/src/main/asciidoc/_chapters/mapreduce.adoc
index 61cff86..bba8cc9 100644
--- a/src/main/asciidoc/_chapters/mapreduce.adoc
+++ b/src/main/asciidoc/_chapters/mapreduce.adoc
@@ -417,8 +417,8 @@ public static class MyMapper extends TableMapper<ImmutableBytesWritable, Put>  {
 
     private static Put resultToPut(ImmutableBytesWritable key, Result result) throws IOException {
       Put put = new Put(key.get());
-      for (KeyValue kv : result.raw()) {
-        put.add(kv);
+      for (Cell cell : result.listCells()) {
+        put.add(cell);
       }
       return put;
     }


[hbase] 03/49: HBASE-22045 Mutable range histogram reports incorrect outliers

Posted by op...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

openinx pushed a commit to branch HBASE-21879
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit 2a9eab9fd3d6aaebfcbccf2ab2ab2dcd545b676f
Author: Abhishek Singh Chouhan <ab...@gmail.com>
AuthorDate: Tue Mar 12 11:24:57 2019 -0700

    HBASE-22045 Mutable range histogram reports incorrect outliers
---
 .../main/java/org/apache/hadoop/metrics2/lib/MutableRangeHistogram.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableRangeHistogram.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableRangeHistogram.java
index 4c800c4..4b73087 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableRangeHistogram.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableRangeHistogram.java
@@ -81,7 +81,7 @@ public abstract class MutableRangeHistogram extends MutableHistogram implements
       priorRange = ranges[i];
       cumNum = val;
     }
-    long val = histogram.getCount();
+    long val = snapshot.getCount();
     if (val - cumNum > 0) {
       metricsRecordBuilder.addCounter(
           Interns.info(name + "_" + rangeType + "_" + ranges[ranges.length - 1] + "-inf", desc),


[hbase] 14/49: HBASE-22056 Unexpected blank line in ClusterConnection

Posted by op...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

openinx pushed a commit to branch HBASE-21879
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit 2a929155a79fd54895b9be5650404efa86c60cca
Author: pingsutw <pi...@gmail.com>
AuthorDate: Mon Mar 18 16:59:31 2019 +0800

    HBASE-22056 Unexpected blank line in ClusterConnection
    
    Signed-off-by: Josh Elser <el...@apache.org>
---
 .../src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java | 2 --
 1 file changed, 2 deletions(-)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java
index 9b2222b..304f832 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java
@@ -1,6 +1,4 @@
 /**
- *
-
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information


[hbase] 23/49: HBASE-21965 Fix failed split and merge transactions that have failed to roll back

Posted by op...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

openinx pushed a commit to branch HBASE-21879
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit 9cde2e9ac08ecb0bf0459b88b3e29c2dc215d607
Author: Jingyun Tian <ti...@gmail.com>
AuthorDate: Fri Mar 22 13:38:53 2019 +0800

    HBASE-21965 Fix failed split and merge transactions that have failed to roll back
---
 .../org/apache/hadoop/hbase/client/HBaseHbck.java  |  16 ++
 .../java/org/apache/hadoop/hbase/client/Hbck.java  |  15 ++
 .../hbase/shaded/protobuf/RequestConverter.java    |   9 ++
 .../src/main/protobuf/Master.proto                 |  16 ++
 .../hadoop/hbase/master/MasterRpcServices.java     | 167 ++++++++++++++++++++
 .../assignment/SplitTableRegionProcedure.java      |  66 ++++----
 .../hbase/regionserver/HRegionFileSystem.java      |   6 +
 .../org/apache/hadoop/hbase/client/TestHbck.java   | 172 +++++++++++++++++++++
 8 files changed, 439 insertions(+), 28 deletions(-)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseHbck.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseHbck.java
index a276017..1963906 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseHbck.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseHbck.java
@@ -19,10 +19,12 @@ package org.apache.hadoop.hbase.client;
 
 import java.io.IOException;
 import java.util.List;
+import java.util.Map;
 import java.util.concurrent.Callable;
 import java.util.stream.Collectors;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
@@ -172,4 +174,18 @@ public class HBaseHbck implements Hbck {
       throw new IOException(se);
     }
   }
+
+  @Override
+  public Map<String, MasterProtos.REGION_ERROR_TYPE>
+      getFailedSplitMergeLegacyRegions(List<TableName> tableNames) throws IOException {
+    try {
+      MasterProtos.GetFailedSplitMergeLegacyRegionsResponse response =
+          this.hbck.getFailedSplitMergeLegacyRegions(rpcControllerFactory.newController(),
+            RequestConverter.toGetFailedSplitMergeLegacyRegionsRequest(tableNames));
+      return response.getErrorsMap();
+    } catch (ServiceException se) {
+      LOG.debug("get failed split/merge legacy regions failed", se);
+      throw new IOException(se);
+    }
+  }
 }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Hbck.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Hbck.java
index e88805c..49fb18f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Hbck.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Hbck.java
@@ -20,12 +20,15 @@ package org.apache.hadoop.hbase.client;
 import java.io.Closeable;
 import java.io.IOException;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.yetus.audience.InterfaceAudience;
 
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos;
 
 /**
  * Hbck fixup tool APIs. Obtain an instance from {@link ClusterConnection#getHbck()} and call
@@ -106,4 +109,16 @@ public interface Hbck extends Abortable, Closeable {
 
   List<Long> scheduleServerCrashProcedure(List<HBaseProtos.ServerName> serverNames)
       throws IOException;
+
+  /**
+   * This method is to get the regions which left by failed split/merge procedures for a certain
+   * table. There are two kinds of region this method will return. One is orphan regions left on FS,
+   * which left because split/merge procedure crashed before updating meta. And the other one is
+   * unassigned split daughter region or merged region, which left because split/merge procedure
+   * crashed before assignment.
+   * @param tableName table to check
+   * @return Map of problematic regions
+   */
+  Map<String, MasterProtos.REGION_ERROR_TYPE>
+      getFailedSplitMergeLegacyRegions(List<TableName> tableName) throws IOException;
 }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
index 36c8fab..e7b6624 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
@@ -1913,6 +1913,15 @@ public final class RequestConverter {
     return b.addAllServerName(serverNames).build();
   }
 
+  public static MasterProtos.GetFailedSplitMergeLegacyRegionsRequest
+      toGetFailedSplitMergeLegacyRegionsRequest(List<TableName> tableNames) {
+    MasterProtos.GetFailedSplitMergeLegacyRegionsRequest.Builder b =
+        MasterProtos.GetFailedSplitMergeLegacyRegionsRequest.newBuilder();
+    List<HBaseProtos.TableName> protoTableNames = tableNames.stream()
+        .map(tableName -> ProtobufUtil.toProtoTableName(tableName)).collect(Collectors.toList());
+    return b.addAllTable(protoTableNames).build();
+  }
+
   private static List<RegionSpecifier> toEncodedRegionNameRegionSpecifiers(
       List<String> encodedRegionNames) {
     return encodedRegionNames.stream().
diff --git a/hbase-protocol-shaded/src/main/protobuf/Master.proto b/hbase-protocol-shaded/src/main/protobuf/Master.proto
index 4ed0ad5..f9fbf08 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Master.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Master.proto
@@ -1093,6 +1093,19 @@ message ScheduleServerCrashProcedureResponse {
   repeated uint64 pid = 1;
 }
 
+message GetFailedSplitMergeLegacyRegionsRequest {
+  repeated TableName table = 1;
+}
+
+enum REGION_ERROR_TYPE {
+  daughter_merged_region_not_online = 0;
+  orphan_region_on_fs = 1;
+}
+
+message GetFailedSplitMergeLegacyRegionsResponse {
+  map<string, REGION_ERROR_TYPE> errors = 1;
+}
+
 service HbckService {
   /** Update state of the table in meta only*/
   rpc SetTableStateInMeta(SetTableStateInMetaRequest)
@@ -1123,4 +1136,7 @@ service HbckService {
   /** Schedule a ServerCrashProcedure to help recover a crash server */
   rpc ScheduleServerCrashProcedure(ScheduleServerCrashProcedureRequest)
     returns(ScheduleServerCrashProcedureResponse);
+
+  rpc getFailedSplitMergeLegacyRegions(GetFailedSplitMergeLegacyRegionsRequest)
+    returns(GetFailedSplitMergeLegacyRegionsResponse);
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index b943000..c396036 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -27,12 +27,14 @@ import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
 import java.util.stream.Collectors;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.ClusterMetricsBuilder;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
@@ -49,6 +51,7 @@ import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.MasterSwitchType;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableState;
@@ -66,7 +69,10 @@ import org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface;
 import org.apache.hadoop.hbase.ipc.RpcServerFactory;
 import org.apache.hadoop.hbase.ipc.RpcServerInterface;
 import org.apache.hadoop.hbase.ipc.ServerRpcController;
+import org.apache.hadoop.hbase.master.assignment.MergeTableRegionsProcedure;
+import org.apache.hadoop.hbase.master.assignment.RegionStateStore;
 import org.apache.hadoop.hbase.master.assignment.RegionStates;
+import org.apache.hadoop.hbase.master.assignment.SplitTableRegionProcedure;
 import org.apache.hadoop.hbase.master.locking.LockProcedure;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil;
@@ -86,6 +92,7 @@ import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
 import org.apache.hadoop.hbase.quotas.QuotaObserverChore;
 import org.apache.hadoop.hbase.quotas.QuotaUtil;
 import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot;
+import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
 import org.apache.hadoop.hbase.regionserver.RSRpcServices;
 import org.apache.hadoop.hbase.regionserver.RpcSchedulerFactory;
 import org.apache.hadoop.hbase.replication.ReplicationException;
@@ -103,8 +110,10 @@ import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
 import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
 import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.PairOfSameType;
 import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -2495,6 +2504,164 @@ public class MasterRpcServices extends RSRpcServices
   }
 
   @Override
+  public MasterProtos.GetFailedSplitMergeLegacyRegionsResponse getFailedSplitMergeLegacyRegions(
+      RpcController controller, MasterProtos.GetFailedSplitMergeLegacyRegionsRequest request)
+      throws ServiceException {
+    List<HBaseProtos.TableName> tables = request.getTableList();
+
+    Map<String, MasterProtos.REGION_ERROR_TYPE> errorRegions = new HashMap<>();
+    try {
+      for (HBaseProtos.TableName tableName : tables) {
+        errorRegions.putAll(getFailedSplitMergeLegacyRegions(ProtobufUtil.toTableName(tableName)));
+      }
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return MasterProtos.GetFailedSplitMergeLegacyRegionsResponse.newBuilder()
+        .putAllErrors(errorRegions).build();
+  }
+
+  private Map<String, MasterProtos.REGION_ERROR_TYPE>
+      getFailedSplitMergeLegacyRegions(TableName tableName) throws IOException {
+    if (!MetaTableAccessor.tableExists(master.getConnection(), tableName)) {
+      throw new IOException("table " + tableName.getNameAsString() + " doesn't exist");
+    }
+    if (!MetaTableAccessor.getTableState(master.getConnection(), tableName).isEnabled()) {
+      throw new IOException(
+          "table " + tableName.getNameAsString() + " is not enabled yet");
+    }
+    final Map<String, MasterProtos.REGION_ERROR_TYPE> problemRegions = new HashMap<>();
+
+    // Case 1. find orphan region on fs
+    // orphan regions may due to a failed split region procedure, which daughter regions are created
+    // then the procedure is aborted. Or merged region is created then the procedure is aborted.
+    List<String> orphanRegions = findOrphanRegionOnFS(tableName);
+    orphanRegions.stream().forEach(
+      region -> problemRegions.put(region, MasterProtos.REGION_ERROR_TYPE.orphan_region_on_fs));
+
+    // Case 2. find unassigned daughter regions or merged regions
+    List<String> unassignedDaughterOrMergedRegions =
+        findUnassignedDaughterOrMergedRegions(tableName);
+    unassignedDaughterOrMergedRegions.stream().forEach(region -> problemRegions.put(region,
+      MasterProtos.REGION_ERROR_TYPE.daughter_merged_region_not_online));
+
+    // if these regions in problemRegions are currently handled by SplitTableRegionProcedure or
+    // MergeTableRegionsProcedure, we should remove them from this map
+    master.getProcedures().stream().filter(p -> !(p.isFinished() || p.isBypass())).forEach(p -> {
+      if (p instanceof SplitTableRegionProcedure) {
+        problemRegions
+            .remove(((SplitTableRegionProcedure) p).getDaughterOneRI().getRegionNameAsString());
+        problemRegions
+            .remove(((SplitTableRegionProcedure) p).getDaughterTwoRI().getRegionNameAsString());
+      } else if (p instanceof MergeTableRegionsProcedure) {
+        problemRegions
+            .remove(((MergeTableRegionsProcedure) p).getMergedRegion().getRegionNameAsString());
+      }
+    });
+
+    // check if regions are still problematic now
+    checkRegionStillProblematic(problemRegions, tableName);
+    return problemRegions;
+  }
+
+
+  private void checkRegionStillProblematic(
+      Map<String, MasterProtos.REGION_ERROR_TYPE> problemRegions, TableName tableName)
+      throws IOException {
+    Iterator<Map.Entry<String, MasterProtos.REGION_ERROR_TYPE>> iterator =
+        problemRegions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<String, MasterProtos.REGION_ERROR_TYPE> entry = iterator.next();
+      Result r = MetaTableAccessor.getRegionResult(master.getConnection(),
+        Bytes.toBytesBinary(entry.getKey()));
+      RegionInfo regionInfo = MetaTableAccessor.getRegionInfo(r);
+      switch (entry.getValue()) {
+        case orphan_region_on_fs:
+          // region is build for this directory, it is not a problematic region any more
+          if (r != null) {
+            problemRegions.remove(regionInfo.getRegionNameAsString());
+          }
+          break;
+        case daughter_merged_region_not_online:
+          RegionState.State state = RegionStateStore.getRegionState(r, 0);
+          if (!state.matches(RegionState.State.CLOSED, RegionState.State.SPLITTING_NEW,
+            RegionState.State.MERGED)) {
+            problemRegions.remove(regionInfo.getRegionNameAsString());
+          }
+          break;
+        default:
+          throw new IOException("there should be no problematic region of this type");
+      }
+    }
+  }
+
+  private List<String> findUnassignedDaughterOrMergedRegions(TableName tableName)
+      throws IOException {
+    Set<String> checkRegions = new HashSet<>();
+    Map<String, RegionState.State> regionStates = new HashMap<>();
+    Map<String, RegionInfo> regionInfos = new HashMap<>();
+
+    MetaTableAccessor.scanMeta(master.getConnection(), tableName,
+      MetaTableAccessor.QueryType.REGION, Integer.MAX_VALUE, r -> {
+        RegionInfo regionInfo = MetaTableAccessor.getRegionInfo(r);
+        regionInfos.put(regionInfo.getRegionNameAsString(), regionInfo);
+        RegionState.State state = RegionStateStore.getRegionState(r, 0);
+        regionStates.put(regionInfo.getEncodedName(), state);
+        if (regionInfo.isSplitParent()) {
+          PairOfSameType<RegionInfo> daughters = MetaTableAccessor.getDaughterRegions(r);
+          checkRegions.add(daughters.getFirst().getRegionNameAsString());
+          checkRegions.add(daughters.getSecond().getRegionNameAsString());
+        } else if (r.getValue(HConstants.CATALOG_FAMILY, HConstants.MERGEA_QUALIFIER) != null
+            || r.getValue(HConstants.CATALOG_FAMILY, HConstants.MERGEB_QUALIFIER) != null) {
+          checkRegions.add(regionInfo.getRegionNameAsString());
+        }
+        return true;
+      });
+
+    // find unassigned merged or split daughter region
+    return checkRegions.stream().map(regionName -> regionInfos.get(regionName))
+        .filter(regionInfo -> !regionInfo.isSplitParent())
+        .filter(regionInfo -> !regionStates.get(regionInfo.getEncodedName())
+            .matches(RegionState.State.OPEN))
+        .map(regionInfo -> regionInfo.getRegionNameAsString()).collect(Collectors.toList());
+  }
+
+  private List<String> findOrphanRegionOnFS(TableName tableName) throws IOException {
+    // get available regions from meta, merged region should be consider available
+    HashSet<String> regionsInMeta = new HashSet<>();
+    MetaTableAccessor.scanMeta(master.getConnection(), tableName,
+      MetaTableAccessor.QueryType.REGION, Integer.MAX_VALUE, r -> {
+        RegionInfo regionInfo = MetaTableAccessor.getRegionInfo(r);
+        regionsInMeta.add(regionInfo.getEncodedName());
+        if (r.getValue(HConstants.CATALOG_FAMILY, HConstants.MERGEA_QUALIFIER) != null
+            || r.getValue(HConstants.CATALOG_FAMILY, HConstants.MERGEB_QUALIFIER) != null) {
+          PairOfSameType<RegionInfo> mergedRegions = MetaTableAccessor.getMergeRegions(r);
+          regionsInMeta.add(mergedRegions.getFirst().getEncodedName());
+          regionsInMeta.add(mergedRegions.getSecond().getEncodedName());
+        }
+        return true;
+      });
+    // get regionInfo from fs
+    Path tableDir = FSUtils.getTableDir(master.getMasterFileSystem().getRootDir(), tableName);
+    FileStatus[] regions =
+        master.getFileSystem().listStatus(tableDir, path -> !path.getName().startsWith("."));
+    HashMap<String, String> regionNames = new HashMap<>();
+    for (FileStatus region : regions) {
+      RegionInfo regionInfo =
+          HRegionFileSystem.loadRegionInfoFileContent(master.getFileSystem(), region.getPath());
+      regionNames.put(regionInfo.getEncodedName(), regionInfo.getRegionNameAsString());
+    }
+    Iterator<Map.Entry<String, String>> regionIterator = regionNames.entrySet().iterator();
+    while (regionIterator.hasNext()) {
+      Map.Entry<String, String> region = regionIterator.next();
+      if (regionsInMeta.contains(region.getKey())) {
+        regionIterator.remove();
+      }
+    }
+    return new ArrayList<>(regionNames.values());
+  }
+
+  @Override
   public SwitchRpcThrottleResponse switchRpcThrottle(RpcController controller,
       SwitchRpcThrottleRequest request) throws ServiceException {
     try {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
index 7b74919..657f397 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
@@ -93,8 +93,8 @@ public class SplitTableRegionProcedure
     extends AbstractStateMachineRegionProcedure<SplitTableRegionState> {
   private static final Logger LOG = LoggerFactory.getLogger(SplitTableRegionProcedure.class);
   private Boolean traceEnabled = null;
-  private RegionInfo daughter_1_RI;
-  private RegionInfo daughter_2_RI;
+  private RegionInfo daughterOneRI;
+  private RegionInfo daughterTwoRI;
   private byte[] bestSplitRow;
   private RegionSplitPolicy splitPolicy;
 
@@ -113,13 +113,13 @@ public class SplitTableRegionProcedure
     checkSplittable(env, regionToSplit, bestSplitRow);
     final TableName table = regionToSplit.getTable();
     final long rid = getDaughterRegionIdTimestamp(regionToSplit);
-    this.daughter_1_RI = RegionInfoBuilder.newBuilder(table)
+    this.daughterOneRI = RegionInfoBuilder.newBuilder(table)
         .setStartKey(regionToSplit.getStartKey())
         .setEndKey(bestSplitRow)
         .setSplit(false)
         .setRegionId(rid)
         .build();
-    this.daughter_2_RI = RegionInfoBuilder.newBuilder(table)
+    this.daughterTwoRI = RegionInfoBuilder.newBuilder(table)
         .setStartKey(bestSplitRow)
         .setEndKey(regionToSplit.getEndKey())
         .setSplit(false)
@@ -140,7 +140,7 @@ public class SplitTableRegionProcedure
   @Override
   protected LockState acquireLock(final MasterProcedureEnv env) {
     if (env.getProcedureScheduler().waitRegions(this, getTableName(), getParentRegion(),
-      daughter_1_RI, daughter_2_RI)) {
+      daughterOneRI, daughterTwoRI)) {
       try {
         LOG.debug(LockState.LOCK_EVENT_WAIT + " " + env.getProcedureScheduler().dumpLocks());
       } catch (IOException e) {
@@ -153,8 +153,18 @@ public class SplitTableRegionProcedure
 
   @Override
   protected void releaseLock(final MasterProcedureEnv env) {
-    env.getProcedureScheduler().wakeRegions(this, getTableName(), getParentRegion(), daughter_1_RI,
-      daughter_2_RI);
+    env.getProcedureScheduler().wakeRegions(this, getTableName(), getParentRegion(), daughterOneRI,
+      daughterTwoRI);
+  }
+
+  @VisibleForTesting
+  public RegionInfo getDaughterOneRI() {
+    return daughterOneRI;
+  }
+
+  @VisibleForTesting
+  public RegionInfo getDaughterTwoRI() {
+    return daughterTwoRI;
   }
 
   /**
@@ -416,8 +426,8 @@ public class SplitTableRegionProcedure
         MasterProcedureProtos.SplitTableRegionStateData.newBuilder()
         .setUserInfo(MasterProcedureUtil.toProtoUserInfo(getUser()))
         .setParentRegionInfo(ProtobufUtil.toRegionInfo(getRegion()))
-        .addChildRegionInfo(ProtobufUtil.toRegionInfo(daughter_1_RI))
-        .addChildRegionInfo(ProtobufUtil.toRegionInfo(daughter_2_RI));
+        .addChildRegionInfo(ProtobufUtil.toRegionInfo(daughterOneRI))
+        .addChildRegionInfo(ProtobufUtil.toRegionInfo(daughterTwoRI));
     serializer.serialize(splitTableRegionMsg.build());
   }
 
@@ -431,8 +441,8 @@ public class SplitTableRegionProcedure
     setUser(MasterProcedureUtil.toUserInfo(splitTableRegionsMsg.getUserInfo()));
     setRegion(ProtobufUtil.toRegionInfo(splitTableRegionsMsg.getParentRegionInfo()));
     assert(splitTableRegionsMsg.getChildRegionInfoCount() == 2);
-    daughter_1_RI = ProtobufUtil.toRegionInfo(splitTableRegionsMsg.getChildRegionInfo(0));
-    daughter_2_RI = ProtobufUtil.toRegionInfo(splitTableRegionsMsg.getChildRegionInfo(1));
+    daughterOneRI = ProtobufUtil.toRegionInfo(splitTableRegionsMsg.getChildRegionInfo(0));
+    daughterTwoRI = ProtobufUtil.toRegionInfo(splitTableRegionsMsg.getChildRegionInfo(1));
   }
 
   @Override
@@ -443,9 +453,9 @@ public class SplitTableRegionProcedure
     sb.append(", parent=");
     sb.append(getParentRegion().getShortNameToLog());
     sb.append(", daughterA=");
-    sb.append(daughter_1_RI.getShortNameToLog());
+    sb.append(daughterOneRI.getShortNameToLog());
     sb.append(", daughterB=");
-    sb.append(daughter_2_RI.getShortNameToLog());
+    sb.append(daughterTwoRI.getShortNameToLog());
   }
 
   private RegionInfo getParentRegion() {
@@ -463,7 +473,7 @@ public class SplitTableRegionProcedure
   }
 
   private byte[] getSplitRow() {
-    return daughter_2_RI.getStartKey();
+    return daughterTwoRI.getStartKey();
   }
 
   private static final State[] EXPECTED_SPLIT_STATES = new State[] { State.OPEN, State.CLOSED };
@@ -595,17 +605,17 @@ public class SplitTableRegionProcedure
     Pair<Integer, Integer> expectedReferences = splitStoreFiles(env, regionFs);
 
     assertReferenceFileCount(fs, expectedReferences.getFirst(),
-      regionFs.getSplitsDir(daughter_1_RI));
+      regionFs.getSplitsDir(daughterOneRI));
     //Move the files from the temporary .splits to the final /table/region directory
-    regionFs.commitDaughterRegion(daughter_1_RI);
+    regionFs.commitDaughterRegion(daughterOneRI);
     assertReferenceFileCount(fs, expectedReferences.getFirst(),
-      new Path(tabledir, daughter_1_RI.getEncodedName()));
+      new Path(tabledir, daughterOneRI.getEncodedName()));
 
     assertReferenceFileCount(fs, expectedReferences.getSecond(),
-      regionFs.getSplitsDir(daughter_2_RI));
-    regionFs.commitDaughterRegion(daughter_2_RI);
+      regionFs.getSplitsDir(daughterTwoRI));
+    regionFs.commitDaughterRegion(daughterTwoRI);
     assertReferenceFileCount(fs, expectedReferences.getSecond(),
-      new Path(tabledir, daughter_2_RI.getEncodedName()));
+      new Path(tabledir, daughterTwoRI.getEncodedName()));
   }
 
   /**
@@ -745,9 +755,9 @@ public class SplitTableRegionProcedure
 
     final byte[] splitRow = getSplitRow();
     final String familyName = Bytes.toString(family);
-    final Path path_first = regionFs.splitStoreFile(this.daughter_1_RI, familyName, sf, splitRow,
+    final Path path_first = regionFs.splitStoreFile(this.daughterOneRI, familyName, sf, splitRow,
         false, splitPolicy);
-    final Path path_second = regionFs.splitStoreFile(this.daughter_2_RI, familyName, sf, splitRow,
+    final Path path_second = regionFs.splitStoreFile(this.daughterTwoRI, familyName, sf, splitRow,
        true, splitPolicy);
     if (LOG.isDebugEnabled()) {
       LOG.debug("pid=" + getProcId() + " splitting complete for store file: " +
@@ -812,7 +822,7 @@ public class SplitTableRegionProcedure
    */
   private void updateMeta(final MasterProcedureEnv env) throws IOException {
     env.getAssignmentManager().markRegionAsSplit(getParentRegion(), getParentRegionServerName(env),
-      daughter_1_RI, daughter_2_RI);
+        daughterOneRI, daughterTwoRI);
   }
 
   /**
@@ -834,7 +844,7 @@ public class SplitTableRegionProcedure
   private void postSplitRegion(final MasterProcedureEnv env) throws IOException {
     final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
     if (cpHost != null) {
-      cpHost.postCompletedSplitRegionAction(daughter_1_RI, daughter_2_RI, getUser());
+      cpHost.postCompletedSplitRegionAction(daughterOneRI, daughterTwoRI, getUser());
     }
   }
 
@@ -852,8 +862,8 @@ public class SplitTableRegionProcedure
   private TransitRegionStateProcedure[] createAssignProcedures(MasterProcedureEnv env)
       throws IOException {
     List<RegionInfo> hris = new ArrayList<RegionInfo>(2);
-    hris.add(daughter_1_RI);
-    hris.add(daughter_2_RI);
+    hris.add(daughterOneRI);
+    hris.add(daughterTwoRI);
     return AssignmentManagerUtil.createAssignProceduresForOpeningNewRegions(env, hris,
       getRegionReplication(env), getParentRegionServerName(env));
   }
@@ -868,9 +878,9 @@ public class SplitTableRegionProcedure
     long maxSequenceId =
       WALSplitter.getMaxRegionSequenceId(walFS, getWALRegionDir(env, getParentRegion()));
     if (maxSequenceId > 0) {
-      WALSplitter.writeRegionSequenceIdFile(walFS, getWALRegionDir(env, daughter_1_RI),
+      WALSplitter.writeRegionSequenceIdFile(walFS, getWALRegionDir(env, daughterOneRI),
           maxSequenceId);
-      WALSplitter.writeRegionSequenceIdFile(walFS, getWALRegionDir(env, daughter_2_RI),
+      WALSplitter.writeRegionSequenceIdFile(walFS, getWALRegionDir(env, daughterTwoRI),
           maxSequenceId);
     }
   }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
index 9666aa5..4d19bd5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
@@ -807,6 +807,12 @@ public class HRegionFileSystem {
     Path mergedRegionTmpDir = this.getMergesDir(mergedRegionInfo);
     // Move the tmp dir in the expected location
     if (mergedRegionTmpDir != null && fs.exists(mergedRegionTmpDir)) {
+
+      // Write HRI to a file in case we need to recover hbase:meta
+      Path regionInfoFile = new Path(mergedRegionTmpDir, REGION_INFO_FILE);
+      byte[] regionInfoContent = getRegionInfoFileContent(regionInfo);
+      writeRegionInfoFileContent(conf, fs, regionInfoFile, regionInfoContent);
+
       if (!fs.rename(mergedRegionTmpDir, regionDir)) {
         throw new IOException("Unable to rename " + mergedRegionTmpDir + " to "
             + regionDir);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHbck.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHbck.java
index 8318757..eb4a2fd 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHbck.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHbck.java
@@ -18,17 +18,32 @@
 package org.apache.hadoop.hbase.client;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
 import java.util.stream.Collectors;
+
+import org.apache.hadoop.hbase.Coprocessor;
+import org.apache.hadoop.hbase.CoprocessorEnvironment;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.coprocessor.MasterCoprocessor;
+import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.MasterObserver;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.master.assignment.MergeTableRegionsProcedure;
+import org.apache.hadoop.hbase.master.assignment.SplitTableRegionProcedure;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.master.procedure.TableProcedureInterface;
 import org.apache.hadoop.hbase.procedure2.Procedure;
@@ -40,6 +55,7 @@ import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.AfterClass;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
@@ -57,6 +73,7 @@ import org.slf4j.LoggerFactory;
 import org.apache.hbase.thirdparty.com.google.common.io.Closeables;
 
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos;
 
 /**
  * Class to test HBaseHbck. Spins up the minicluster once at test start and then takes it down
@@ -102,6 +119,12 @@ public class TestHbck {
     TEST_UTIL.createMultiRegionTable(TABLE_NAME, Bytes.toBytes("family1"), 5);
     procExec = TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor();
     ASYNC_CONN = ConnectionFactory.createAsyncConnection(TEST_UTIL.getConfiguration()).get();
+    TEST_UTIL.getHBaseCluster().getMaster().getMasterCoprocessorHost().load(
+      FailingMergeAfterMetaUpdatedMasterObserver.class, Coprocessor.PRIORITY_USER,
+      TEST_UTIL.getHBaseCluster().getMaster().getConfiguration());
+    TEST_UTIL.getHBaseCluster().getMaster().getMasterCoprocessorHost().load(
+      FailingSplitAfterMetaUpdatedMasterObserver.class, Coprocessor.PRIORITY_USER,
+      TEST_UTIL.getHBaseCluster().getMaster().getConfiguration());
   }
 
   @AfterClass
@@ -205,6 +228,101 @@ public class TestHbck {
   }
 
   @Test
+  public void testRecoverMergeAfterMetaUpdated() throws Exception {
+    String testTable = async ? "mergeTestAsync" : "mergeTestSync";
+    TEST_UTIL.createMultiRegionTable(TableName.valueOf(testTable), Bytes.toBytes("family1"), 5);
+    TEST_UTIL.loadTable(TEST_UTIL.getConnection().getTable(TableName.valueOf(testTable)),
+      Bytes.toBytes("family1"), true);
+    HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
+    Hbck hbck = getHbck();
+    FailingMergeAfterMetaUpdatedMasterObserver observer = master.getMasterCoprocessorHost()
+        .findCoprocessor(FailingMergeAfterMetaUpdatedMasterObserver.class);
+    try (Admin admin = TEST_UTIL.getConnection().getAdmin()) {
+      List<RegionInfo> regions = admin.getRegions(TableName.valueOf(testTable));
+      admin.mergeRegionsAsync(regions.get(0).getRegionName(), regions.get(1).getRegionName(), true);
+      assertNotNull(observer);
+      observer.latch.await(5000, TimeUnit.MILLISECONDS);
+      Map<String, MasterProtos.REGION_ERROR_TYPE> result =
+          hbck.getFailedSplitMergeLegacyRegions(Arrays.asList(TableName.valueOf(testTable)));
+      Assert.assertEquals(0, result.size());
+      Optional<Procedure<?>> procedure = TEST_UTIL.getHBaseCluster().getMaster().getProcedures()
+          .stream().filter(p -> p instanceof MergeTableRegionsProcedure).findAny();
+      Assert.assertTrue(procedure.isPresent());
+      hbck.bypassProcedure(Arrays.asList(procedure.get().getProcId()), 5, true, false);
+      result = hbck.getFailedSplitMergeLegacyRegions(Arrays.asList(TableName.valueOf(testTable)));
+      Assert.assertEquals(1, result.size());
+      hbck.assigns(Arrays.asList(result.keySet().toArray(new String[0])).stream()
+          .map(regionName -> regionName.split("\\.")[1]).collect(Collectors.toList()));
+      ProcedureTestingUtility.waitNoProcedureRunning(master.getMasterProcedureExecutor());
+      // now the state should be fixed
+      result = hbck.getFailedSplitMergeLegacyRegions(Arrays.asList(TableName.valueOf(testTable)));
+      Assert.assertEquals(0, result.size());
+    } catch (InterruptedException ie) {
+      throw new IOException(ie);
+    } finally {
+      observer.resetLatch();
+    }
+  }
+
+  @Test
+  public void testRecoverSplitAfterMetaUpdated() throws Exception {
+    String testTable = async ? "splitTestAsync" : "splitTestSync";
+    TEST_UTIL.createMultiRegionTable(TableName.valueOf(testTable), Bytes.toBytes("family1"), 5);
+    HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
+    Hbck hbck = getHbck();
+    FailingSplitAfterMetaUpdatedMasterObserver observer = master.getMasterCoprocessorHost()
+        .findCoprocessor(FailingSplitAfterMetaUpdatedMasterObserver.class);
+    assertNotNull(observer);
+    try (Admin admin = TEST_UTIL.getConnection().getAdmin()) {
+      byte[] splitKey = Bytes.toBytes("bcd");
+      admin.split(TableName.valueOf(testTable), splitKey);
+      observer.latch.await(5000, TimeUnit.MILLISECONDS);
+      Map<String, MasterProtos.REGION_ERROR_TYPE> result =
+          hbck.getFailedSplitMergeLegacyRegions(Arrays.asList(TableName.valueOf(testTable)));
+      // since there is a split procedure work on the region, thus this check should return a empty
+      // map.
+      Assert.assertEquals(0, result.size());
+      Optional<Procedure<?>> procedure = TEST_UTIL.getHBaseCluster().getMaster().getProcedures()
+          .stream().filter(p -> p instanceof SplitTableRegionProcedure).findAny();
+      Assert.assertTrue(procedure.isPresent());
+      hbck.bypassProcedure(Arrays.asList(procedure.get().getProcId()), 5, true, false);
+      result = hbck.getFailedSplitMergeLegacyRegions(Arrays.asList(TableName.valueOf(testTable)));
+      Assert.assertEquals(2, result.size());
+      hbck.assigns(Arrays.asList(result.keySet().toArray(new String[0])).stream()
+          .map(regionName -> regionName.split("\\.")[1]).collect(Collectors.toList()));
+      ProcedureTestingUtility.waitNoProcedureRunning(master.getMasterProcedureExecutor());
+      // now the state should be fixed
+      result = hbck.getFailedSplitMergeLegacyRegions(Arrays.asList(TableName.valueOf(testTable)));
+      Assert.assertEquals(0, result.size());
+
+      //split one of the daughter region again
+      observer.resetLatch();
+      byte[] splitKey2 = Bytes.toBytes("bcde");
+
+      admin.split(TableName.valueOf(testTable), splitKey2);
+      observer.latch.await(5000, TimeUnit.MILLISECONDS);
+
+      procedure = TEST_UTIL.getHBaseCluster().getMaster().getProcedures()
+          .stream().filter(p -> p instanceof SplitTableRegionProcedure).findAny();
+      Assert.assertTrue(procedure.isPresent());
+      hbck.bypassProcedure(Arrays.asList(procedure.get().getProcId()), 5, true, false);
+      result = hbck.getFailedSplitMergeLegacyRegions(Arrays.asList(TableName.valueOf(testTable)));
+      Assert.assertEquals(2, result.size());
+      hbck.assigns(Arrays.asList(result.keySet().toArray(new String[0])).stream()
+          .map(regionName -> regionName.split("\\.")[1]).collect(Collectors.toList()));
+      ProcedureTestingUtility.waitNoProcedureRunning(master.getMasterProcedureExecutor());
+      // now the state should be fixed
+      result = hbck.getFailedSplitMergeLegacyRegions(Arrays.asList(TableName.valueOf(testTable)));
+      Assert.assertEquals(0, result.size());
+    } catch (InterruptedException ie) {
+      throw new IOException(ie);
+    } finally {
+      observer.resetLatch();
+    }
+  }
+
+
+  @Test
   public void testScheduleSCP() throws Exception {
     HRegionServer testRs = TEST_UTIL.getRSForFirstRegionInTable(TABLE_NAME);
     TEST_UTIL.loadTable(TEST_UTIL.getConnection().getTable(TABLE_NAME), Bytes.toBytes("family1"),
@@ -223,6 +341,60 @@ public class TestHbck {
     waitOnPids(pids);
   }
 
+  public static class FailingSplitAfterMetaUpdatedMasterObserver
+      implements MasterCoprocessor, MasterObserver {
+    public volatile CountDownLatch latch;
+
+    @Override
+    public void start(CoprocessorEnvironment e) throws IOException {
+      resetLatch();
+    }
+
+    @Override
+    public Optional<MasterObserver> getMasterObserver() {
+      return Optional.of(this);
+    }
+
+    @Override
+    public void preSplitRegionAfterMETAAction(ObserverContext<MasterCoprocessorEnvironment> ctx)
+        throws IOException {
+      LOG.info("I'm here");
+      latch.countDown();
+      throw new IOException("this procedure will fail at here forever");
+    }
+
+    public void resetLatch() {
+      this.latch = new CountDownLatch(1);
+    }
+  }
+
+  public static class FailingMergeAfterMetaUpdatedMasterObserver
+      implements MasterCoprocessor, MasterObserver {
+    public volatile CountDownLatch latch;
+
+    @Override
+    public void start(CoprocessorEnvironment e) throws IOException {
+      resetLatch();
+    }
+
+    @Override
+    public Optional<MasterObserver> getMasterObserver() {
+      return Optional.of(this);
+    }
+
+    public void resetLatch() {
+      this.latch = new CountDownLatch(1);
+    }
+
+    @Override
+    public void postMergeRegionsCommitAction(
+        final ObserverContext<MasterCoprocessorEnvironment> ctx, final RegionInfo[] regionsToMerge,
+        final RegionInfo mergedRegion) throws IOException {
+      latch.countDown();
+      throw new IOException("this procedure will fail at here forever");
+    }
+  }
+
   private void waitOnPids(List<Long> pids) {
     TEST_UTIL.waitFor(60000, () -> pids.stream().allMatch(procExec::isFinished));
   }


[hbase] 18/49: HBASE-22077 Expose sleep time as CLI argument of IntergationTestBackupRestore (Tamas Adami)

Posted by op...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

openinx pushed a commit to branch HBASE-21879
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit 9617279dc13e725a8fea2710192423b7ff1999e4
Author: Josh Elser <el...@apache.org>
AuthorDate: Thu Mar 21 10:13:44 2019 -0400

    HBASE-22077 Expose sleep time as CLI argument of IntergationTestBackupRestore (Tamas Adami)
---
 .../hadoop/hbase/IntegrationTestBackupRestore.java | 24 +++++++++++++++-------
 1 file changed, 17 insertions(+), 7 deletions(-)

diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestBackupRestore.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestBackupRestore.java
index 35d526d..ff6ae17 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestBackupRestore.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestBackupRestore.java
@@ -55,6 +55,8 @@ import org.apache.hadoop.util.ToolRunner;
 import org.apache.hbase.thirdparty.com.google.common.base.MoreObjects;
 import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 import org.apache.hbase.thirdparty.com.google.common.util.concurrent.Uninterruptibles;
+import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
+
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -63,8 +65,6 @@ import org.junit.experimental.categories.Category;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
-import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
 
 /**
  * An integration test to detect regressions in HBASE-7912. Create
@@ -335,7 +335,9 @@ public class IntegrationTestBackupRestore extends IntegrationTestBase {
    */
   protected boolean checkSucceeded(String backupId) throws IOException {
     BackupInfo status = getBackupInfo(backupId);
-    if (status == null) return false;
+    if (status == null) {
+      return false;
+    }
     return status.getState() == BackupState.COMPLETE;
   }
 
@@ -416,7 +418,8 @@ public class IntegrationTestBackupRestore extends IntegrationTestBase {
       "Total number iterations." + " Default: " + DEFAULT_NUM_ITERATIONS);
     addOptWithArg(NUMBER_OF_TABLES_KEY,
       "Total number of tables in the test." + " Default: " + DEFAULT_NUMBER_OF_TABLES);
-
+    addOptWithArg(SLEEP_TIME_KEY, "Sleep time of chaos monkey in ms " +
+            "to restart random region server. Default: " + SLEEP_TIME_DEFAULT);
   }
 
   @Override
@@ -435,10 +438,17 @@ public class IntegrationTestBackupRestore extends IntegrationTestBase {
       Integer.toString(DEFAULT_NUM_ITERATIONS)));
     numTables = Integer.parseInt(cmd.getOptionValue(NUMBER_OF_TABLES_KEY,
       Integer.toString(DEFAULT_NUMBER_OF_TABLES)));
+    sleepTime = Long.parseLong(cmd.getOptionValue(SLEEP_TIME_KEY,
+      Long.toString(SLEEP_TIME_DEFAULT)));
+
 
-    LOG.info(MoreObjects.toStringHelper("Parsed Options").
-      add(REGION_COUNT_KEY, regionsCountPerServer)
-      .add(REGIONSERVER_COUNT_KEY, regionServerCount).add(ROWS_PER_ITERATION_KEY, rowsInIteration)
+    LOG.info(MoreObjects.toStringHelper("Parsed Options")
+      .add(REGION_COUNT_KEY, regionsCountPerServer)
+      .add(REGIONSERVER_COUNT_KEY, regionServerCount)
+      .add(ROWS_PER_ITERATION_KEY, rowsInIteration)
+      .add(NUM_ITERATIONS_KEY, numIterations)
+      .add(NUMBER_OF_TABLES_KEY, numTables)
+      .add(SLEEP_TIME_KEY, sleepTime)
       .toString());
   }
 


[hbase] 22/49: HBASE-21991: Fix MetaMetrics issues - [Race condition, Faulty remove logic], few improvements

Posted by op...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

openinx pushed a commit to branch HBASE-21879
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit 82042863f704e390854260d1cc66190203c60b5c
Author: Sakthi <sa...@gmail.com>
AuthorDate: Sat Mar 9 16:09:34 2019 -0800

    HBASE-21991: Fix MetaMetrics issues - [Race condition, Faulty remove logic], few improvements
    
    Signed-off-by: Xu Cang <xu...@apache.org>
---
 .../hadoop/hbase/coprocessor/MetaTableMetrics.java | 118 ++++++++++++---------
 .../apache/hadoop/hbase/util/LossyCounting.java    |  11 +-
 .../hbase/coprocessor/TestMetaTableMetrics.java    |  99 +++++++++++++++++
 .../hadoop/hbase/util/TestLossyCounting.java       |  10 +-
 4 files changed, 177 insertions(+), 61 deletions(-)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MetaTableMetrics.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MetaTableMetrics.java
index d08bae6..d542d2f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MetaTableMetrics.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MetaTableMetrics.java
@@ -50,8 +50,8 @@ public class MetaTableMetrics implements RegionCoprocessor {
 
   private ExampleRegionObserverMeta observer;
   private Map<String, Optional<Metric>> requestsMap;
-  private RegionCoprocessorEnvironment regionCoprocessorEnv;
-  private LossyCounting clientMetricsLossyCounting;
+  private MetricRegistry registry;
+  private LossyCounting clientMetricsLossyCounting, regionMetricsLossyCounting;
   private boolean active = false;
 
   enum MetaTableOps {
@@ -94,11 +94,11 @@ public class MetaTableMetrics implements RegionCoprocessor {
       if (!active || !isMetaTableOp(e)) {
         return;
       }
-      tableMetricRegisterAndMark(e, row);
-      clientMetricRegisterAndMark(e);
-      regionMetricRegisterAndMark(e, row);
-      opMetricRegisterAndMark(e, row);
-      opWithClientMetricRegisterAndMark(e, row);
+      tableMetricRegisterAndMark(row);
+      clientMetricRegisterAndMark();
+      regionMetricRegisterAndMark(row);
+      opMetricRegisterAndMark(row);
+      opWithClientMetricRegisterAndMark(row);
     }
 
     private void markMeterIfPresent(String requestMeter) {
@@ -106,19 +106,18 @@ public class MetaTableMetrics implements RegionCoprocessor {
         return;
       }
 
-      if (requestsMap.containsKey(requestMeter) && requestsMap.get(requestMeter).isPresent()) {
-        Meter metric = (Meter) requestsMap.get(requestMeter).get();
+      Optional<Metric> optionalMetric = requestsMap.get(requestMeter);
+      if (optionalMetric != null && optionalMetric.isPresent()) {
+        Meter metric = (Meter) optionalMetric.get();
         metric.mark();
       }
     }
 
-    private void registerMeterIfNotPresent(ObserverContext<RegionCoprocessorEnvironment> e,
-        String requestMeter) {
+    private void registerMeterIfNotPresent(String requestMeter) {
       if (requestMeter.isEmpty()) {
         return;
       }
       if (!requestsMap.containsKey(requestMeter)) {
-        MetricRegistry registry = regionCoprocessorEnv.getMetricRegistryForRegionServer();
         registry.meter(requestMeter);
         requestsMap.put(requestMeter, registry.get(requestMeter));
       }
@@ -129,32 +128,36 @@ public class MetaTableMetrics implements RegionCoprocessor {
      * By using lossy count to maintain meters, at most 7 / e meters will be kept  (e is error rate)
      * e.g. when e is 0.02 by default, at most 350 Clients request metrics will be kept
      *      also, all kept elements have frequency higher than e * N. (N is total count)
-     * @param e Region coprocessor environment
      * @param requestMeter meter to be registered
      * @param lossyCounting lossyCounting object for one type of meters.
      */
-    private void registerLossyCountingMeterIfNotPresent(
-        ObserverContext<RegionCoprocessorEnvironment> e,
-        String requestMeter, LossyCounting lossyCounting) {
+    private void registerLossyCountingMeterIfNotPresent(String requestMeter,
+        LossyCounting lossyCounting) {
+
       if (requestMeter.isEmpty()) {
         return;
       }
-      Set<String> metersToBeRemoved = lossyCounting.addByOne(requestMeter);
-      if(!requestsMap.containsKey(requestMeter) && metersToBeRemoved.contains(requestMeter)){
-        for(String meter: metersToBeRemoved) {
-          //cleanup requestsMap according swept data from lossy count;
+      synchronized (lossyCounting) {
+        Set<String> metersToBeRemoved = lossyCounting.addByOne(requestMeter);
+
+        boolean isNewMeter = !requestsMap.containsKey(requestMeter);
+        boolean requestMeterRemoved = metersToBeRemoved.contains(requestMeter);
+        if (isNewMeter) {
+          if (requestMeterRemoved) {
+            // if the new metric is swept off by lossyCounting then don't add in the map
+            metersToBeRemoved.remove(requestMeter);
+          } else {
+            // else register the new metric and add in the map
+            registry.meter(requestMeter);
+            requestsMap.put(requestMeter, registry.get(requestMeter));
+          }
+        }
+
+        for (String meter : metersToBeRemoved) {
+          //cleanup requestsMap according to the swept data from lossy count;
           requestsMap.remove(meter);
-          MetricRegistry registry = regionCoprocessorEnv.getMetricRegistryForRegionServer();
           registry.remove(meter);
         }
-        // newly added meter is swept by lossy counting cleanup. No need to put it into requestsMap.
-        return;
-      }
-
-      if (!requestsMap.containsKey(requestMeter)) {
-        MetricRegistry registry = regionCoprocessorEnv.getMetricRegistryForRegionServer();
-        registry.meter(requestMeter);
-        requestsMap.put(requestMeter, registry.get(requestMeter));
       }
     }
 
@@ -191,49 +194,59 @@ public class MetaTableMetrics implements RegionCoprocessor {
           .equals(e.getEnvironment().getRegionInfo().getTable());
     }
 
-    private void clientMetricRegisterAndMark(ObserverContext<RegionCoprocessorEnvironment> e) {
+    private void clientMetricRegisterAndMark() {
       // Mark client metric
       String clientIP = RpcServer.getRemoteIp() != null ? RpcServer.getRemoteIp().toString() : "";
-
+      if (clientIP == null || clientIP.isEmpty()) {
+        return;
+      }
       String clientRequestMeter = clientRequestMeterName(clientIP);
-      registerLossyCountingMeterIfNotPresent(e, clientRequestMeter, clientMetricsLossyCounting);
+      registerLossyCountingMeterIfNotPresent(clientRequestMeter, clientMetricsLossyCounting);
       markMeterIfPresent(clientRequestMeter);
     }
 
-    private void tableMetricRegisterAndMark(ObserverContext<RegionCoprocessorEnvironment> e,
-        Row op) {
+    private void tableMetricRegisterAndMark(Row op) {
       // Mark table metric
       String tableName = getTableNameFromOp(op);
+      if (tableName == null || tableName.isEmpty()) {
+        return;
+      }
       String tableRequestMeter = tableMeterName(tableName);
-      registerAndMarkMeterIfNotPresent(e, tableRequestMeter);
+      registerAndMarkMeterIfNotPresent(tableRequestMeter);
     }
 
-    private void regionMetricRegisterAndMark(ObserverContext<RegionCoprocessorEnvironment> e,
-        Row op) {
+    private void regionMetricRegisterAndMark(Row op) {
       // Mark region metric
       String regionId = getRegionIdFromOp(op);
+      if (regionId == null || regionId.isEmpty()) {
+        return;
+      }
       String regionRequestMeter = regionMeterName(regionId);
-      registerAndMarkMeterIfNotPresent(e, regionRequestMeter);
+      registerLossyCountingMeterIfNotPresent(regionRequestMeter, regionMetricsLossyCounting);
+      markMeterIfPresent(regionRequestMeter);
     }
 
-    private void opMetricRegisterAndMark(ObserverContext<RegionCoprocessorEnvironment> e,
-        Row op) {
+    private void opMetricRegisterAndMark(Row op) {
       // Mark access type ["get", "put", "delete"] metric
       String opMeterName = opMeterName(op);
-      registerAndMarkMeterIfNotPresent(e, opMeterName);
+      if (opMeterName == null || opMeterName.isEmpty()) {
+        return;
+      }
+      registerAndMarkMeterIfNotPresent(opMeterName);
     }
 
-    private void opWithClientMetricRegisterAndMark(ObserverContext<RegionCoprocessorEnvironment> e,
-        Object op) {
+    private void opWithClientMetricRegisterAndMark(Object op) {
       // // Mark client + access type metric
       String opWithClientMeterName = opWithClientMeterName(op);
-      registerAndMarkMeterIfNotPresent(e, opWithClientMeterName);
+      if (opWithClientMeterName == null || opWithClientMeterName.isEmpty()) {
+        return;
+      }
+      registerAndMarkMeterIfNotPresent(opWithClientMeterName);
     }
 
     // Helper function to register and mark meter if not present
-    private void registerAndMarkMeterIfNotPresent(ObserverContext<RegionCoprocessorEnvironment> e,
-        String name) {
-      registerMeterIfNotPresent(e, name);
+    private void registerAndMarkMeterIfNotPresent(String name) {
+      registerMeterIfNotPresent(name);
       markMeterIfPresent(name);
     }
 
@@ -291,12 +304,12 @@ public class MetaTableMetrics implements RegionCoprocessor {
       if (clientIP.isEmpty()) {
         return "";
       }
-      return String.format("MetaTable_client_%s_request", clientIP);
+      return String.format("MetaTable_client_%s_lossy_request", clientIP);
     }
 
     private String regionMeterName(String regionId) {
       // Extract meter name containing the region ID
-      return String.format("MetaTable_region_%s_request", regionId);
+      return String.format("MetaTable_region_%s_lossy_request", regionId);
     }
   }
 
@@ -312,9 +325,11 @@ public class MetaTableMetrics implements RegionCoprocessor {
         && ((RegionCoprocessorEnvironment) env).getRegionInfo().getTable() != null
         && ((RegionCoprocessorEnvironment) env).getRegionInfo().getTable()
           .equals(TableName.META_TABLE_NAME)) {
-      regionCoprocessorEnv = (RegionCoprocessorEnvironment) env;
+      RegionCoprocessorEnvironment regionCoprocessorEnv = (RegionCoprocessorEnvironment) env;
+      registry = regionCoprocessorEnv.getMetricRegistryForRegionServer();
       requestsMap = new ConcurrentHashMap<>();
-      clientMetricsLossyCounting = new LossyCounting();
+      clientMetricsLossyCounting = new LossyCounting("clientMetaMetrics");
+      regionMetricsLossyCounting = new LossyCounting("regionMetaMetrics");
       // only be active mode when this region holds meta table.
       active = true;
     }
@@ -324,7 +339,6 @@ public class MetaTableMetrics implements RegionCoprocessor {
   public void stop(CoprocessorEnvironment env) throws IOException {
     // since meta region can move around, clear stale metrics when stop.
     if (requestsMap != null) {
-      MetricRegistry registry = regionCoprocessorEnv.getMetricRegistryForRegionServer();
       for (String meterName : requestsMap.keySet()) {
         registry.remove(meterName);
       }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/LossyCounting.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/LossyCounting.java
index 839bb90..d9d84e5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/LossyCounting.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/LossyCounting.java
@@ -51,9 +51,11 @@ public class LossyCounting {
   private double errorRate;
   private Map<String, Integer> data;
   private long totalDataCount;
+  private String name;
 
-  public LossyCounting(double errorRate) {
+  public LossyCounting(double errorRate, String name) {
     this.errorRate = errorRate;
+    this.name = name;
     if (errorRate < 0.0 || errorRate > 1.0) {
       throw new IllegalArgumentException(" Lossy Counting error rate should be within range [0,1]");
     }
@@ -64,8 +66,9 @@ public class LossyCounting {
     calculateCurrentTerm();
   }
 
-  public LossyCounting() {
-    this(HBaseConfiguration.create().getDouble(HConstants.DEFAULT_LOSSY_COUNTING_ERROR_RATE, 0.02));
+  public LossyCounting(String name) {
+    this(HBaseConfiguration.create().getDouble(HConstants.DEFAULT_LOSSY_COUNTING_ERROR_RATE, 0.02),
+        name);
   }
 
   public Set<String> addByOne(String key) {
@@ -93,7 +96,7 @@ public class LossyCounting {
     for(String key : dataToBeSwept) {
       data.remove(key);
     }
-    LOG.debug(String.format("Swept %d elements.", dataToBeSwept.size()));
+    LOG.trace(String.format("%s swept %d elements.", name, dataToBeSwept.size()));
     return dataToBeSwept;
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMetaTableMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMetaTableMetrics.java
index bbbeb9e..82ce709 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMetaTableMetrics.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMetaTableMetrics.java
@@ -13,6 +13,8 @@ package org.apache.hadoop.hbase.coprocessor;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -38,6 +40,7 @@ import org.apache.hadoop.hbase.JMXListener;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
@@ -73,6 +76,11 @@ public class TestMetaTableMetrics {
   private static Configuration conf = null;
   private static int connectorPort = 61120;
 
+  final byte[] cf = Bytes.toBytes("info");
+  final byte[] col = Bytes.toBytes("any");
+  byte[] tablename;
+  final int nthreads = 20;
+
   @BeforeClass
   public static void setupBeforeClass() throws Exception {
 
@@ -224,4 +232,95 @@ public class TestMetaTableMetrics {
     assertEquals(5L, putWithClientMetricsCount);
   }
 
+  @Test(timeout = 30000)
+  public void testConcurrentAccess() {
+    try {
+      tablename = Bytes.toBytes("hbase:meta");
+      int numRows = 3000;
+      int numRowsInTableBefore = UTIL.countRows(TableName.valueOf(tablename));
+      putData(numRows);
+      Thread.sleep(2000);
+      int numRowsInTableAfter = UTIL.countRows(TableName.valueOf(tablename));
+      assertTrue(numRowsInTableAfter >= numRowsInTableBefore + numRows);
+      getData(numRows);
+    } catch (InterruptedException e) {
+      LOG.info("Caught InterruptedException while testConcurrentAccess: " + e.getMessage());
+      fail();
+    } catch (IOException e) {
+      LOG.info("Caught IOException while testConcurrentAccess: " + e.getMessage());
+      fail();
+    }
+  }
+
+  public void putData(int nrows) throws InterruptedException {
+    LOG.info(String.format("Putting %d rows in hbase:meta", nrows));
+    Thread[] threads = new Thread[nthreads];
+    for (int i = 1; i <= nthreads; i++) {
+      threads[i - 1] = new PutThread(1, nrows);
+    }
+    startThreadsAndWaitToJoin(threads);
+  }
+
+  public void getData(int nrows) throws InterruptedException {
+    LOG.info(String.format("Getting %d rows from hbase:meta", nrows));
+    Thread[] threads = new Thread[nthreads];
+    for (int i = 1; i <= nthreads; i++) {
+      threads[i - 1] = new GetThread(1, nrows);
+    }
+    startThreadsAndWaitToJoin(threads);
+  }
+
+  private void startThreadsAndWaitToJoin(Thread[] threads) throws InterruptedException {
+    for (int i = 1; i <= nthreads; i++) {
+      threads[i - 1].start();
+    }
+    for (int i = 1; i <= nthreads; i++) {
+      threads[i - 1].join();
+    }
+  }
+
+  class PutThread extends Thread {
+    int start;
+    int end;
+
+    public PutThread(int start, int end) {
+      this.start = start;
+      this.end = end;
+    }
+
+    @Override
+    public void run() {
+      try (Table table = UTIL.getConnection().getTable(TableName.valueOf(tablename))) {
+        for (int i = start; i <= end; i++) {
+          Put p = new Put(Bytes.toBytes(String.format("tableName,rowKey%d,region%d", i, i)));
+          p.addColumn(cf, col, Bytes.toBytes("Value" + i));
+          table.put(p);
+        }
+      } catch (IOException e) {
+        LOG.info("Caught IOException while PutThread operation: " + e.getMessage());
+      }
+    }
+  }
+
+  class GetThread extends Thread {
+    int start;
+    int end;
+
+    public GetThread(int start, int end) {
+      this.start = start;
+      this.end = end;
+    }
+
+    @Override
+    public void run() {
+      try (Table table = UTIL.getConnection().getTable(TableName.valueOf(tablename))) {
+        for (int i = start; i <= end; i++) {
+          Get get = new Get(Bytes.toBytes(String.format("tableName,rowKey%d,region%d", i, i)));
+          table.get(get);
+        }
+      } catch (IOException e) {
+        LOG.info("Caught IOException while GetThread operation: " + e.getMessage());
+      }
+    }
+  }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestLossyCounting.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestLossyCounting.java
index 11758be..e4f1939 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestLossyCounting.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestLossyCounting.java
@@ -38,15 +38,15 @@ public class TestLossyCounting {
 
   @Test
   public void testBucketSize() {
-    LossyCounting lossyCounting = new LossyCounting(0.01);
+    LossyCounting lossyCounting = new LossyCounting(0.01, "testBucketSize");
     assertEquals(100L, lossyCounting.getBucketSize());
-    LossyCounting lossyCounting2 = new LossyCounting();
+    LossyCounting lossyCounting2 = new LossyCounting("testBucketSize2");
     assertEquals(50L, lossyCounting2.getBucketSize());
   }
 
   @Test
   public void testAddByOne() {
-    LossyCounting lossyCounting = new LossyCounting(0.01);
+    LossyCounting lossyCounting = new LossyCounting(0.01, "testAddByOne");
     for(int i = 0; i < 100; i++){
       String key = "" + i;
       lossyCounting.addByOne(key);
@@ -60,7 +60,7 @@ public class TestLossyCounting {
 
   @Test
   public void testSweep1() {
-    LossyCounting lossyCounting = new LossyCounting(0.01);
+    LossyCounting lossyCounting = new LossyCounting(0.01, "testSweep1");
     for(int i = 0; i < 400; i++){
       String key = "" + i;
       lossyCounting.addByOne(key);
@@ -71,7 +71,7 @@ public class TestLossyCounting {
 
   @Test
   public void testSweep2() {
-    LossyCounting lossyCounting = new LossyCounting(0.1);
+    LossyCounting lossyCounting = new LossyCounting(0.1, "testSweep2");
     for(int i = 0; i < 10; i++){
       String key = "" + i;
       lossyCounting.addByOne(key);


[hbase] 33/49: HBASE-22074 Should use procedure store to persist the state in reportRegionStateTransition

Posted by op...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

openinx pushed a commit to branch HBASE-21879
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit ae0b6b6e04e9d65d28de0443ac2b27eaf63c655b
Author: zhangduo <zh...@apache.org>
AuthorDate: Tue Mar 26 11:28:41 2019 +0800

    HBASE-22074 Should use procedure store to persist the state in reportRegionStateTransition
---
 .../hadoop/hbase/shaded/protobuf/ProtobufUtil.java |  32 +--
 .../hbase/shaded/protobuf/RequestConverter.java    |  28 +--
 .../src/main/protobuf/Admin.proto                  |   3 +-
 .../src/main/protobuf/MasterProcedure.proto        |  11 ++
 .../src/main/protobuf/RegionServerStatus.proto     |   1 +
 .../hbase/master/assignment/AssignProcedure.java   |   6 +-
 .../hbase/master/assignment/AssignmentManager.java |  12 +-
 .../master/assignment/CloseRegionProcedure.java    |  26 ++-
 .../master/assignment/OpenRegionProcedure.java     |  60 +++++-
 .../assignment/RegionRemoteProcedureBase.java      | 219 ++++++++++++++++-----
 .../assignment/TransitRegionStateProcedure.java    | 208 ++++---------------
 .../hbase/master/assignment/UnassignProcedure.java |   4 +-
 .../master/procedure/RSProcedureDispatcher.java    |  54 ++---
 .../hadoop/hbase/regionserver/HRegionServer.java   |  10 +-
 .../hadoop/hbase/regionserver/RSRpcServices.java   |   8 +-
 .../hbase/regionserver/RegionServerServices.java   |  31 ++-
 .../hadoop/hbase/regionserver/SplitRequest.java    |   5 +-
 .../regionserver/handler/AssignRegionHandler.java  |  14 +-
 .../regionserver/handler/CloseRegionHandler.java   |   3 +-
 .../regionserver/handler/OpenRegionHandler.java    |  14 +-
 .../handler/UnassignRegionHandler.java             |  15 +-
 .../assignment/TestAssignmentManagerBase.java      |   7 +
 .../assignment/TestCloseRegionWhileRSCrash.java    |  14 +-
 .../assignment/TestOpenRegionProcedureHang.java    | 209 ++++++++++++++++++++
 .../procedure/TestServerRemoteProcedure.java       |  16 +-
 25 files changed, 636 insertions(+), 374 deletions(-)

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 15a8c8a..336c59c 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
@@ -3003,28 +3003,32 @@ public final class ProtobufUtil {
    }
 
   /**
-    * Create a CloseRegionRequest for a given region name
-    *
-    * @param regionName the name of the region to close
-    * @return a CloseRegionRequest
-    */
-   public static CloseRegionRequest buildCloseRegionRequest(ServerName server,
-       final byte[] regionName) {
-     return ProtobufUtil.buildCloseRegionRequest(server, regionName, null);
-   }
+   * Create a CloseRegionRequest for a given region name
+   * @param regionName the name of the region to close
+   * @return a CloseRegionRequest
+   */
+  public static CloseRegionRequest buildCloseRegionRequest(ServerName server, byte[] regionName) {
+    return ProtobufUtil.buildCloseRegionRequest(server, regionName, null);
+  }
+
+  public static CloseRegionRequest buildCloseRegionRequest(ServerName server, byte[] regionName,
+      ServerName destinationServer) {
+    return buildCloseRegionRequest(server, regionName, destinationServer, -1);
+  }
 
-  public static CloseRegionRequest buildCloseRegionRequest(ServerName server,
-    final byte[] regionName, ServerName destinationServer) {
+  public static CloseRegionRequest buildCloseRegionRequest(ServerName server, byte[] regionName,
+      ServerName destinationServer, long closeProcId) {
     CloseRegionRequest.Builder builder = CloseRegionRequest.newBuilder();
-    RegionSpecifier region = RequestConverter.buildRegionSpecifier(
-      RegionSpecifierType.REGION_NAME, regionName);
+    RegionSpecifier region =
+      RequestConverter.buildRegionSpecifier(RegionSpecifierType.REGION_NAME, regionName);
     builder.setRegion(region);
-    if (destinationServer != null){
+    if (destinationServer != null) {
       builder.setDestinationServer(toServerName(destinationServer));
     }
     if (server != null) {
       builder.setServerStartCode(server.getStartcode());
     }
+    builder.setCloseProcId(closeProcId);
     return builder.build();
   }
 
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
index 36c8fab..0c58d4b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
@@ -941,27 +941,6 @@ public final class RequestConverter {
   }
 
   /**
-   * Create a protocol buffer OpenRegionRequest to open a list of regions
-   * @param server the serverName for the RPC
-   * @param regionOpenInfos info of a list of regions to open
-   * @return a protocol buffer OpenRegionRequest
-   */
-  public static OpenRegionRequest buildOpenRegionRequest(ServerName server,
-      final List<Pair<RegionInfo, List<ServerName>>> regionOpenInfos) {
-    OpenRegionRequest.Builder builder = OpenRegionRequest.newBuilder();
-    for (Pair<RegionInfo, List<ServerName>> regionOpenInfo : regionOpenInfos) {
-      builder.addOpenInfo(buildRegionOpenInfo(regionOpenInfo.getFirst(),
-        regionOpenInfo.getSecond()));
-    }
-    if (server != null) {
-      builder.setServerStartCode(server.getStartcode());
-    }
-    // send the master's wall clock time as well, so that the RS can refer to it
-    builder.setMasterSystemTime(EnvironmentEdgeManager.currentTime());
-    return builder.build();
-  }
-
-  /**
    * Create a protocol buffer OpenRegionRequest for a given region
    * @param server the serverName for the RPC
    * @param region the region to open
@@ -971,7 +950,7 @@ public final class RequestConverter {
   public static OpenRegionRequest buildOpenRegionRequest(ServerName server,
       final RegionInfo region, List<ServerName> favoredNodes) {
     OpenRegionRequest.Builder builder = OpenRegionRequest.newBuilder();
-    builder.addOpenInfo(buildRegionOpenInfo(region, favoredNodes));
+    builder.addOpenInfo(buildRegionOpenInfo(region, favoredNodes, -1L));
     if (server != null) {
       builder.setServerStartCode(server.getStartcode());
     }
@@ -1622,8 +1601,8 @@ public final class RequestConverter {
   /**
    * Create a RegionOpenInfo based on given region info and version of offline node
    */
-  public static RegionOpenInfo buildRegionOpenInfo(
-      final RegionInfo region, final List<ServerName> favoredNodes) {
+  public static RegionOpenInfo buildRegionOpenInfo(RegionInfo region, List<ServerName> favoredNodes,
+      long openProcId) {
     RegionOpenInfo.Builder builder = RegionOpenInfo.newBuilder();
     builder.setRegion(ProtobufUtil.toRegionInfo(region));
     if (favoredNodes != null) {
@@ -1631,6 +1610,7 @@ public final class RequestConverter {
         builder.addFavoredNodes(ProtobufUtil.toServerName(server));
       }
     }
+    builder.setOpenProcId(openProcId);
     return builder.build();
   }
 
diff --git a/hbase-protocol-shaded/src/main/protobuf/Admin.proto b/hbase-protocol-shaded/src/main/protobuf/Admin.proto
index c622d58..85b9113 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Admin.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Admin.proto
@@ -88,6 +88,7 @@ message OpenRegionRequest {
     repeated ServerName favored_nodes = 3;
     // open region for distributedLogReplay
     // optional bool DEPRECATED_openForDistributedLogReplay = 4;
+    optional int64 open_proc_id = 5 [default = -1];
   }
 }
 
@@ -102,7 +103,6 @@ message OpenRegionResponse {
 }
 
 message WarmupRegionRequest {
-
     required RegionInfo regionInfo = 1;
 }
 
@@ -120,6 +120,7 @@ message CloseRegionRequest {
   optional ServerName destination_server = 4;
   // the intended server for this RPC.
   optional uint64 serverStartCode = 5;
+  optional int64 close_proc_id = 6 [default = -1];
 }
 
 message CloseRegionResponse {
diff --git a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
index 64ac398..d5a390c 100644
--- a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
@@ -28,6 +28,7 @@ import "HBase.proto";
 import "RPC.proto";
 import "Snapshot.proto";
 import "Replication.proto";
+import "RegionServerStatus.proto";
 
 // ============================================================================
 //  WARNING - Compatibility rules
@@ -548,9 +549,19 @@ message RegionStateTransitionStateData {
   required bool force_new_plan = 3;
 }
 
+enum RegionRemoteProcedureBaseState {
+  REGION_REMOTE_PROCEDURE_DISPATCH = 1;
+  REGION_REMOTE_PROCEDURE_REPORT_SUCCEED = 2;
+  REGION_REMOTE_PROCEDURE_DISPATCH_FAIL = 3;
+  REGION_REMOTE_PROCEDURE_SERVER_CRASH = 4;
+}
+
 message RegionRemoteProcedureBaseStateData {
   required RegionInfo region = 1;
   required ServerName target_server = 2;
+  required RegionRemoteProcedureBaseState state = 3;
+  optional RegionStateTransition.TransitionCode transition_code = 4;
+  optional int64 seq_id = 5;
 }
 
 message OpenRegionProcedureStateData {
diff --git a/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto b/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto
index 002432a..0137cb1 100644
--- a/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto
@@ -96,6 +96,7 @@ message RegionStateTransition {
   /** For newly opened region, the open seq num is needed */
   optional uint64 open_seq_num = 3;
 
+  repeated int64 proc_id = 4;
   enum TransitionCode {
     OPENED = 0;
     FAILED_OPEN = 1;
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignProcedure.java
index 33a3545..35510d6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignProcedure.java
@@ -21,7 +21,6 @@ import java.io.IOException;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.exceptions.UnexpectedStateException;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
-import org.apache.hadoop.hbase.master.procedure.RSProcedureDispatcher.RegionOpenOperation;
 import org.apache.hadoop.hbase.procedure2.ProcedureMetrics;
 import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
 import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
@@ -38,7 +37,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProto
  * @deprecated Do not use any more.
  * @see TransitRegionStateProcedure
  */
-// TODO: Add being able to assign a region to open read-only.
 @Deprecated
 @InterfaceAudience.Private
 public class AssignProcedure extends RegionTransitionProcedure {
@@ -121,9 +119,7 @@ public class AssignProcedure extends RegionTransitionProcedure {
   @Override
   public RemoteOperation remoteCallBuild(final MasterProcedureEnv env,
       final ServerName serverName) {
-    assert serverName.equals(getRegionState(env).getRegionLocation());
-    return new RegionOpenOperation(this, getRegionInfo(),
-      env.getAssignmentManager().getFavoredNodes(getRegionInfo()), false);
+    return null;
   }
 
   @Override
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
index 2d0c3be..5e43637 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
@@ -835,8 +835,10 @@ public class AssignmentManager {
         case CLOSED:
           assert transition.getRegionInfoCount() == 1 : transition;
           final RegionInfo hri = ProtobufUtil.toRegionInfo(transition.getRegionInfo(0));
+          long procId =
+            transition.getProcIdCount() > 0 ? transition.getProcId(0) : Procedure.NO_PROC_ID;
           updateRegionTransition(serverName, transition.getTransitionCode(), hri,
-            transition.hasOpenSeqNum() ? transition.getOpenSeqNum() : HConstants.NO_SEQNUM);
+            transition.hasOpenSeqNum() ? transition.getOpenSeqNum() : HConstants.NO_SEQNUM, procId);
           break;
         case READY_TO_SPLIT:
         case SPLIT:
@@ -903,7 +905,7 @@ public class AssignmentManager {
   }
 
   private void updateRegionTransition(ServerName serverName, TransitionCode state,
-      RegionInfo regionInfo, long seqId) throws IOException {
+      RegionInfo regionInfo, long seqId, long procId) throws IOException {
     checkMetaLoaded(regionInfo);
 
     RegionStateNode regionNode = regionStates.getRegionStateNode(regionInfo);
@@ -919,7 +921,7 @@ public class AssignmentManager {
     ServerStateNode serverNode = regionStates.getOrCreateServer(serverName);
     regionNode.lock();
     try {
-      if (!reportTransition(regionNode, serverNode, state, seqId)) {
+      if (!reportTransition(regionNode, serverNode, state, seqId, procId)) {
         // Don't log WARN if shutting down cluster; during shutdown. Avoid the below messages:
         // 2018-08-13 10:45:10,551 WARN ...AssignmentManager: No matching procedure found for
         // rit=OPEN, location=ve0538.halxg.cloudera.com,16020,1533493000958,
@@ -941,14 +943,14 @@ public class AssignmentManager {
   }
 
   private boolean reportTransition(RegionStateNode regionNode, ServerStateNode serverNode,
-      TransitionCode state, long seqId) throws IOException {
+      TransitionCode state, long seqId, long procId) throws IOException {
     ServerName serverName = serverNode.getServerName();
     TransitRegionStateProcedure proc = regionNode.getProcedure();
     if (proc == null) {
       return false;
     }
     proc.reportTransition(master.getMasterProcedureExecutor().getEnvironment(), regionNode,
-      serverName, state, seqId);
+      serverName, state, seqId, procId);
     return true;
   }
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/CloseRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/CloseRegionProcedure.java
index f867e96..4fd60f0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/CloseRegionProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/CloseRegionProcedure.java
@@ -20,15 +20,17 @@ package org.apache.hadoop.hbase.master.assignment;
 import java.io.IOException;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.client.RegionInfo;
-import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.exceptions.UnexpectedStateException;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.master.procedure.RSProcedureDispatcher.RegionCloseOperation;
 import org.apache.hadoop.hbase.procedure2.ProcedureMetrics;
 import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
 import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteOperation;
 import org.apache.yetus.audience.InterfaceAudience;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CloseRegionProcedureStateData;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
 
 /**
  * The remote procedure used to close a region.
@@ -46,9 +48,9 @@ public class CloseRegionProcedure extends RegionRemoteProcedureBase {
     super();
   }
 
-  public CloseRegionProcedure(RegionInfo region, ServerName targetServer,
-      ServerName assignCandidate) {
-    super(region, targetServer);
+  public CloseRegionProcedure(TransitRegionStateProcedure parent, RegionInfo region,
+      ServerName targetServer, ServerName assignCandidate) {
+    super(parent, region, targetServer);
     this.assignCandidate = assignCandidate;
   }
 
@@ -59,7 +61,7 @@ public class CloseRegionProcedure extends RegionRemoteProcedureBase {
 
   @Override
   public RemoteOperation remoteCallBuild(MasterProcedureEnv env, ServerName remote) {
-    return new RegionCloseOperation(this, region, assignCandidate);
+    return new RegionCloseOperation(this, region, getProcId(), assignCandidate);
   }
 
   @Override
@@ -88,7 +90,17 @@ public class CloseRegionProcedure extends RegionRemoteProcedureBase {
   }
 
   @Override
-  protected boolean shouldDispatch(RegionStateNode regionNode) {
-    return regionNode.isInState(RegionState.State.CLOSING);
+  protected void reportTransition(RegionStateNode regionNode, TransitionCode transitionCode,
+      long seqId) throws IOException {
+    if (transitionCode != TransitionCode.CLOSED) {
+      throw new UnexpectedStateException("Received report unexpected " + transitionCode +
+        " transition, " + regionNode.toShortString() + ", " + this + ", expected CLOSED.");
+    }
+  }
+
+  @Override
+  protected void updateTransition(MasterProcedureEnv env, RegionStateNode regionNode,
+      TransitionCode transitionCode, long seqId) throws IOException {
+    env.getAssignmentManager().regionClosed(regionNode, true);
   }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/OpenRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/OpenRegionProcedure.java
index 4b3a976..579b757 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/OpenRegionProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/OpenRegionProcedure.java
@@ -20,15 +20,18 @@ package org.apache.hadoop.hbase.master.assignment;
 import java.io.IOException;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.client.RegionInfo;
-import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.exceptions.UnexpectedStateException;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.master.procedure.RSProcedureDispatcher.RegionOpenOperation;
 import org.apache.hadoop.hbase.procedure2.ProcedureMetrics;
 import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
 import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteOperation;
 import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.OpenRegionProcedureStateData;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
 
 /**
  * The remote procedure used to open a region.
@@ -36,12 +39,15 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.O
 @InterfaceAudience.Private
 public class OpenRegionProcedure extends RegionRemoteProcedureBase {
 
+  private static final Logger LOG = LoggerFactory.getLogger(OpenRegionProcedure.class);
+
   public OpenRegionProcedure() {
     super();
   }
 
-  public OpenRegionProcedure(RegionInfo region, ServerName targetServer) {
-    super(region, targetServer);
+  public OpenRegionProcedure(TransitRegionStateProcedure parent, RegionInfo region,
+      ServerName targetServer) {
+    super(parent, region, targetServer);
   }
 
   @Override
@@ -51,8 +57,7 @@ public class OpenRegionProcedure extends RegionRemoteProcedureBase {
 
   @Override
   public RemoteOperation remoteCallBuild(MasterProcedureEnv env, ServerName remote) {
-    return new RegionOpenOperation(this, region, env.getAssignmentManager().getFavoredNodes(region),
-      false);
+    return new RegionOpenOperation(this, region, getProcId());
   }
 
   @Override
@@ -73,7 +78,48 @@ public class OpenRegionProcedure extends RegionRemoteProcedureBase {
   }
 
   @Override
-  protected boolean shouldDispatch(RegionStateNode regionNode) {
-    return regionNode.isInState(RegionState.State.OPENING);
+  protected void reportTransition(RegionStateNode regionNode, TransitionCode transitionCode,
+      long seqId) throws IOException {
+    switch (transitionCode) {
+      case OPENED:
+        // this is the openSeqNum
+        if (seqId < 0) {
+          throw new UnexpectedStateException("Received report unexpected " + TransitionCode.OPENED +
+            " transition openSeqNum=" + seqId + ", " + regionNode + ", proc=" + this);
+        }
+        break;
+      case FAILED_OPEN:
+        // nothing to check
+        break;
+      default:
+        throw new UnexpectedStateException(
+          "Received report unexpected " + transitionCode + " transition, " +
+            regionNode.toShortString() + ", " + this + ", expected OPENED or FAILED_OPEN.");
+    }
+  }
+
+  @Override
+  protected void updateTransition(MasterProcedureEnv env, RegionStateNode regionNode,
+      TransitionCode transitionCode, long openSeqNum) throws IOException {
+    switch (transitionCode) {
+      case OPENED:
+        if (openSeqNum < regionNode.getOpenSeqNum()) {
+          LOG.warn(
+            "Received report {} transition from {} for {}, pid={} but the new openSeqNum {}" +
+              " is less than the current one {}, ignoring...",
+            transitionCode, targetServer, regionNode, getProcId(), openSeqNum,
+            regionNode.getOpenSeqNum());
+        } else {
+          regionNode.setOpenSeqNum(openSeqNum);
+        }
+        env.getAssignmentManager().regionOpened(regionNode);
+        break;
+      case FAILED_OPEN:
+        env.getAssignmentManager().regionFailedOpen(regionNode, false);
+        break;
+      default:
+        throw new UnexpectedStateException("Unexpected transition code: " + transitionCode);
+    }
+
   }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionRemoteProcedureBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionRemoteProcedureBase.java
index f6d3a2e..4a6f375 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionRemoteProcedureBase.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionRemoteProcedureBase.java
@@ -18,9 +18,11 @@
 package org.apache.hadoop.hbase.master.assignment;
 
 import java.io.IOException;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.exceptions.UnexpectedStateException;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.master.procedure.TableProcedureInterface;
 import org.apache.hadoop.hbase.procedure2.FailedRemoteDispatchException;
@@ -28,6 +30,7 @@ import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
 import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
 import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
+import org.apache.hadoop.hbase.procedure2.ProcedureUtil;
 import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
 import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteProcedure;
 import org.apache.hadoop.hbase.procedure2.RemoteProcedureException;
@@ -36,7 +39,10 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RegionRemoteProcedureBaseState;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RegionRemoteProcedureBaseStateData;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
 
 /**
  * The base class for the remote procedures used to open/close a region.
@@ -53,16 +59,25 @@ public abstract class RegionRemoteProcedureBase extends Procedure<MasterProcedur
 
   protected RegionInfo region;
 
-  private ServerName targetServer;
+  protected ServerName targetServer;
 
-  private boolean dispatched;
+  private RegionRemoteProcedureBaseState state =
+    RegionRemoteProcedureBaseState.REGION_REMOTE_PROCEDURE_DISPATCH;
+
+  private TransitionCode transitionCode;
+
+  private long seqId;
+
+  private int attempt;
 
   protected RegionRemoteProcedureBase() {
   }
 
-  protected RegionRemoteProcedureBase(RegionInfo region, ServerName targetServer) {
+  protected RegionRemoteProcedureBase(TransitRegionStateProcedure parent, RegionInfo region,
+      ServerName targetServer) {
     this.region = region;
     this.targetServer = targetServer;
+    parent.attachRemoteProc(this);
   }
 
   @Override
@@ -86,22 +101,26 @@ public abstract class RegionRemoteProcedureBase extends Procedure<MasterProcedur
     RegionStateNode regionNode = getRegionNode(env);
     regionNode.lock();
     try {
-      LOG.warn("The remote operation {} for region {} to server {} failed", this, region,
-        targetServer, exception);
-      // This could happen as the RSProcedureDispatcher and dead server processor are executed in
-      // different threads. It is possible that we have already scheduled SCP for the targetServer
-      // and woken up this procedure, and assigned the region to another RS, and then the
-      // RSProcedureDispatcher notices that the targetServer is dead so it can not send the request
-      // out and call remoteCallFailed, which makes us arrive here, especially that if the target
-      // machine is completely down, which means you can only receive a ConnectionTimeout after a
-      // very long time(depends on the timeout settings and in HBase usually it will be at least 15
-      // seconds, or even 1 minute). So here we need to check whether we are stilling waiting on the
-      // given event, if not, this means that we have already been woken up so do not wake it up
-      // again.
-      if (!regionNode.getProcedureEvent().wakeIfSuspended(env.getProcedureScheduler(), this)) {
-        LOG.warn("{} is not waiting on the event for region {}, targer server = {}, ignore.", this,
-          region, targetServer);
+      if (!env.getMasterServices().getServerManager().isServerOnline(remote)) {
+        // the SCP will interrupt us, give up
+        LOG.debug("{} for region {}, targetServer {} is dead, SCP will interrupt us, give up", this,
+          regionNode, remote);
+        return;
       }
+      if (state != RegionRemoteProcedureBaseState.REGION_REMOTE_PROCEDURE_DISPATCH) {
+        // not sure how can this happen but anyway let's add a check here to avoid waking the wrong
+        // procedure...
+        LOG.warn("{} for region {}, targetServer={} has already been woken up, ignore", this,
+          regionNode, remote);
+        return;
+      }
+      LOG.warn("The remote operation {} for region {} to server {} failed", this, regionNode,
+        remote, exception);
+      // It is OK to not persist the state here, as we do not need to change the region state if the
+      // remote call is failed. If the master crashed before we actually execute the procedure and
+      // persist the new state, it is fine to retry on the same target server again.
+      state = RegionRemoteProcedureBaseState.REGION_REMOTE_PROCEDURE_DISPATCH_FAIL;
+      regionNode.getProcedureEvent().wake(env.getProcedureScheduler());
     } finally {
       regionNode.unlock();
     }
@@ -133,46 +152,127 @@ public abstract class RegionRemoteProcedureBase extends Procedure<MasterProcedur
     return false;
   }
 
-  /**
-   * Check whether we still need to make the call to RS.
-   * <p/>
-   * This could happen when master restarts. Since we do not know whether a request has already been
-   * sent to the region server after we add a remote operation to the dispatcher, so the safe way is
-   * to not persist the dispatched field and try to add the remote operation again. But it is
-   * possible that we do have already sent the request to region server and it has also sent back
-   * the response, so here we need to check the region state, if it is not in the expecting state,
-   * we should give up, otherwise we may hang for ever, as the region server will just ignore
-   * redundant calls.
-   */
-  protected abstract boolean shouldDispatch(RegionStateNode regionNode);
+  // do some checks to see if the report is valid, without actually updating meta.
+  protected abstract void reportTransition(RegionStateNode regionNode,
+      TransitionCode transitionCode, long seqId) throws IOException;
+
+  // A bit strange but the procedure store will throw RuntimeException if we can not persist the
+  // state, so upper layer should take care of this...
+  private void persistAndWake(MasterProcedureEnv env, RegionStateNode regionNode) {
+    env.getMasterServices().getMasterProcedureExecutor().getStore().update(this);
+    regionNode.getProcedureEvent().wake(env.getProcedureScheduler());
+  }
+
+  // should be called with RegionStateNode locked, to avoid race with the execute method below
+  void reportTransition(MasterProcedureEnv env, RegionStateNode regionNode, ServerName serverName,
+      TransitionCode transitionCode, long seqId) throws IOException {
+    if (state != RegionRemoteProcedureBaseState.REGION_REMOTE_PROCEDURE_DISPATCH) {
+      // should be a retry
+      return;
+    }
+    if (!targetServer.equals(serverName)) {
+      throw new UnexpectedStateException("Received report from " + serverName + ", expected " +
+        targetServer + ", " + regionNode + ", proc=" + this);
+    }
+    reportTransition(regionNode, transitionCode, seqId);
+    // this state means we have received the report from RS, does not mean the result is fine, as we
+    // may received a FAILED_OPEN.
+    this.state = RegionRemoteProcedureBaseState.REGION_REMOTE_PROCEDURE_REPORT_SUCCEED;
+    this.transitionCode = transitionCode;
+    this.seqId = seqId;
+    // Persist the transition code and openSeqNum(if provided).
+    // We should not update the hbase:meta directly as this may cause races when master restarts,
+    // as the old active master may incorrectly report back to RS and cause the new master to hang
+    // on a OpenRegionProcedure forever. See HBASE-22060 and HBASE-22074 for more details.
+    boolean succ = false;
+    try {
+      persistAndWake(env, regionNode);
+      succ = true;
+    } finally {
+      if (!succ) {
+        this.state = RegionRemoteProcedureBaseState.REGION_REMOTE_PROCEDURE_DISPATCH;
+        this.transitionCode = null;
+        this.seqId = HConstants.NO_SEQNUM;
+      }
+    }
+  }
+
+  void serverCrashed(MasterProcedureEnv env, RegionStateNode regionNode, ServerName serverName) {
+    if (state != RegionRemoteProcedureBaseState.REGION_REMOTE_PROCEDURE_DISPATCH) {
+      // should be a retry
+      return;
+    }
+    this.state = RegionRemoteProcedureBaseState.REGION_REMOTE_PROCEDURE_SERVER_CRASH;
+    boolean succ = false;
+    try {
+      persistAndWake(env, regionNode);
+      succ = true;
+    } finally {
+      if (!succ) {
+        this.state = RegionRemoteProcedureBaseState.REGION_REMOTE_PROCEDURE_DISPATCH;
+      }
+    }
+  }
+
+  private TransitRegionStateProcedure getParent(MasterProcedureEnv env) {
+    return (TransitRegionStateProcedure) env.getMasterServices().getMasterProcedureExecutor()
+      .getProcedure(getParentProcId());
+  }
+
+  private void unattach(MasterProcedureEnv env) {
+    getParent(env).unattachRemoteProc(this);
+  }
+
+  // actually update the state to meta
+  protected abstract void updateTransition(MasterProcedureEnv env, RegionStateNode regionNode,
+      TransitionCode transitionCode, long seqId) throws IOException;
 
   @Override
   protected Procedure<MasterProcedureEnv>[] execute(MasterProcedureEnv env)
       throws ProcedureYieldException, ProcedureSuspendedException, InterruptedException {
-    if (dispatched) {
-      // we are done, the parent procedure will check whether we are succeeded.
-      return null;
-    }
     RegionStateNode regionNode = getRegionNode(env);
     regionNode.lock();
     try {
-      if (!shouldDispatch(regionNode)) {
-        return null;
+      switch (state) {
+        case REGION_REMOTE_PROCEDURE_DISPATCH: {
+          // The code which wakes us up also needs to lock the RSN so here we do not need to
+          // synchronize
+          // on the event.
+          ProcedureEvent<?> event = regionNode.getProcedureEvent();
+          try {
+            env.getRemoteDispatcher().addOperationToNode(targetServer, this);
+          } catch (FailedRemoteDispatchException e) {
+            LOG.warn("Can not add remote operation {} for region {} to server {}, this usually " +
+              "because the server is alread dead, give up and mark the procedure as complete, " +
+              "the parent procedure will take care of this.", this, region, targetServer, e);
+            unattach(env);
+            return null;
+          }
+          event.suspend();
+          event.suspendIfNotReady(this);
+          throw new ProcedureSuspendedException();
+        }
+        case REGION_REMOTE_PROCEDURE_REPORT_SUCCEED:
+          updateTransition(env, regionNode, transitionCode, seqId);
+          unattach(env);
+          return null;
+        case REGION_REMOTE_PROCEDURE_DISPATCH_FAIL:
+          // the remote call is failed so we do not need to change the region state, just return.
+          unattach(env);
+          return null;
+        case REGION_REMOTE_PROCEDURE_SERVER_CRASH:
+          env.getAssignmentManager().regionClosed(regionNode, false);
+          unattach(env);
+          return null;
+        default:
+          throw new IllegalStateException("Unknown state: " + state);
       }
-      // The code which wakes us up also needs to lock the RSN so here we do not need to synchronize
-      // on the event.
-      ProcedureEvent<?> event = regionNode.getProcedureEvent();
-      try {
-        env.getRemoteDispatcher().addOperationToNode(targetServer, this);
-      } catch (FailedRemoteDispatchException e) {
-        LOG.warn("Can not add remote operation {} for region {} to server {}, this usually " +
-          "because the server is alread dead, give up and mark the procedure as complete, " +
-          "the parent procedure will take care of this.", this, region, targetServer, e);
-        return null;
-      }
-      dispatched = true;
-      event.suspend();
-      event.suspendIfNotReady(this);
+    } catch (IOException e) {
+      long backoff = ProcedureUtil.getBackoffTimeMs(this.attempt++);
+      LOG.warn("Failed updating meta, suspend {}secs {}; {};", backoff / 1000, this, regionNode, e);
+      setTimeout(Math.toIntExact(backoff));
+      setState(ProcedureProtos.ProcedureState.WAITING_TIMEOUT);
+      skipPersistence();
       throw new ProcedureSuspendedException();
     } finally {
       regionNode.unlock();
@@ -186,9 +286,14 @@ public abstract class RegionRemoteProcedureBase extends Procedure<MasterProcedur
 
   @Override
   protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException {
-    serializer.serialize(
+    RegionRemoteProcedureBaseStateData.Builder builder =
       RegionRemoteProcedureBaseStateData.newBuilder().setRegion(ProtobufUtil.toRegionInfo(region))
-        .setTargetServer(ProtobufUtil.toServerName(targetServer)).build());
+        .setTargetServer(ProtobufUtil.toServerName(targetServer)).setState(state);
+    if (transitionCode != null) {
+      builder.setTransitionCode(transitionCode);
+      builder.setSeqId(seqId);
+    }
+    serializer.serialize(builder.build());
   }
 
   @Override
@@ -197,5 +302,15 @@ public abstract class RegionRemoteProcedureBase extends Procedure<MasterProcedur
       serializer.deserialize(RegionRemoteProcedureBaseStateData.class);
     region = ProtobufUtil.toRegionInfo(data.getRegion());
     targetServer = ProtobufUtil.toServerName(data.getTargetServer());
+    state = data.getState();
+    if (data.hasTransitionCode()) {
+      transitionCode = data.getTransitionCode();
+      seqId = data.getSeqId();
+    }
+  }
+
+  @Override
+  protected void afterReplay(MasterProcedureEnv env) {
+    getParent(env).attachRemoteProc(this);
   }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/TransitRegionStateProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/TransitRegionStateProcedure.java
index d3429b5..1be7a9a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/TransitRegionStateProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/TransitRegionStateProcedure.java
@@ -17,9 +17,6 @@
  */
 package org.apache.hadoop.hbase.master.assignment;
 
-import static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RegionStateTransitionState.REGION_STATE_TRANSITION_CONFIRM_CLOSED;
-import static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RegionStateTransitionState.REGION_STATE_TRANSITION_CONFIRM_OPENED;
-
 import edu.umd.cs.findbugs.annotations.Nullable;
 import java.io.IOException;
 import org.apache.hadoop.hbase.HBaseIOException;
@@ -28,7 +25,6 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
 import org.apache.hadoop.hbase.client.RetriesExhaustedException;
-import org.apache.hadoop.hbase.exceptions.UnexpectedStateException;
 import org.apache.hadoop.hbase.master.MetricsAssignmentManager;
 import org.apache.hadoop.hbase.master.RegionState.State;
 import org.apache.hadoop.hbase.master.procedure.AbstractStateMachineRegionProcedure;
@@ -121,6 +117,8 @@ public class TransitRegionStateProcedure
 
   private int attempt;
 
+  private RegionRemoteProcedureBase remoteProc;
+
   public TransitRegionStateProcedure() {
   }
 
@@ -143,6 +141,7 @@ public class TransitRegionStateProcedure
         throw new IllegalArgumentException("Unknown TransitionType: " + type);
     }
   }
+
   @VisibleForTesting
   protected TransitRegionStateProcedure(MasterProcedureEnv env, RegionInfo hri,
       ServerName assignCandidate, boolean forceNewPlan, TransitionType type) {
@@ -204,21 +203,18 @@ public class TransitRegionStateProcedure
       return;
     }
     env.getAssignmentManager().regionOpening(regionNode);
-    addChildProcedure(new OpenRegionProcedure(getRegion(), loc));
+    addChildProcedure(new OpenRegionProcedure(this, getRegion(), loc));
     setNextState(RegionStateTransitionState.REGION_STATE_TRANSITION_CONFIRM_OPENED);
   }
 
   private Flow confirmOpened(MasterProcedureEnv env, RegionStateNode regionNode)
       throws IOException {
-    // notice that, for normal case, if we successfully opened a region, we will not arrive here, as
-    // in reportTransition we will call unsetProcedure, and in executeFromState we will return
-    // directly. But if the master is crashed before we finish the procedure, then next time we will
-    // arrive here. So we still need to add code for normal cases.
     if (regionNode.isInState(State.OPEN)) {
       attempt = 0;
       if (lastState == RegionStateTransitionState.REGION_STATE_TRANSITION_CONFIRM_OPENED) {
         // we are the last state, finish
         regionNode.unsetProcedure(this);
+        ServerCrashProcedure.updateProgress(env, getParentProcId());
         return Flow.NO_MORE_STATE;
       }
       // It is possible that we arrive here but confirm opened is not the last state, for example,
@@ -250,8 +246,8 @@ public class TransitRegionStateProcedure
     if (regionNode.isInState(State.OPEN, State.CLOSING, State.MERGING, State.SPLITTING)) {
       // this is the normal case
       env.getAssignmentManager().regionClosing(regionNode);
-      addChildProcedure(
-        new CloseRegionProcedure(getRegion(), regionNode.getRegionLocation(), assignCandidate));
+      addChildProcedure(new CloseRegionProcedure(this, getRegion(), regionNode.getRegionLocation(),
+        assignCandidate));
       setNextState(RegionStateTransitionState.REGION_STATE_TRANSITION_CONFIRM_CLOSED);
     } else {
       forceNewPlan = true;
@@ -262,10 +258,6 @@ public class TransitRegionStateProcedure
 
   private Flow confirmClosed(MasterProcedureEnv env, RegionStateNode regionNode)
       throws IOException {
-    // notice that, for normal case, if we successfully opened a region, we will not arrive here, as
-    // in reportTransition we will call unsetProcedure, and in executeFromState we will return
-    // directly. But if the master is crashed before we finish the procedure, then next time we will
-    // arrive here. So we still need to add code for normal cases.
     if (regionNode.isInState(State.CLOSED)) {
       attempt = 0;
       if (lastState == RegionStateTransitionState.REGION_STATE_TRANSITION_CONFIRM_CLOSED) {
@@ -324,14 +316,6 @@ public class TransitRegionStateProcedure
   protected Flow executeFromState(MasterProcedureEnv env, RegionStateTransitionState state)
       throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException {
     RegionStateNode regionNode = getRegionStateNode(env);
-    if (regionNode.getProcedure() != this) {
-      // This is possible, and is the normal case, as we will call unsetProcedure in
-      // reportTransition, this means we have already done
-      // This is because that, when we mark the region as OPENED or CLOSED, then all the works
-      // should have already been done, and logically we could have another TRSP scheduled for this
-      // region immediately(think of a RS crash at the point...).
-      return Flow.NO_MORE_STATE;
-    }
     try {
       switch (state) {
         case REGION_STATE_TRANSITION_GET_ASSIGN_CANDIDATE:
@@ -373,171 +357,47 @@ public class TransitRegionStateProcedure
     return false; // 'false' means that this procedure handled the timeout
   }
 
-  private boolean isOpening(RegionStateNode regionNode, ServerName serverName,
-      TransitionCode code) {
-    if (!regionNode.isInState(State.OPENING)) {
-      LOG.warn("Received report {} transition from {} for {}, pid={}, but the region is not in" +
-        " OPENING state, should be a retry, ignore", code, serverName, regionNode, getProcId());
-      return false;
-    }
-    if (getCurrentState() != REGION_STATE_TRANSITION_CONFIRM_OPENED) {
+  // Should be called with RegionStateNode locked
+  public void reportTransition(MasterProcedureEnv env, RegionStateNode regionNode,
+      ServerName serverName, TransitionCode code, long seqId, long procId) throws IOException {
+    if (remoteProc == null) {
       LOG.warn(
-        "Received report {} transition from {} for {}, pid={}," +
-          " but the TRSP is not in {} state, should be a retry, ignore",
-        code, serverName, regionNode, getProcId(), REGION_STATE_TRANSITION_CONFIRM_OPENED);
-      return false;
-    }
-    return true;
-  }
-
-  private void reportTransitionOpen(MasterProcedureEnv env, RegionStateNode regionNode,
-      ServerName serverName, long openSeqNum) throws IOException {
-    if (!isOpening(regionNode, serverName, TransitionCode.OPENED)) {
+        "There is no outstanding remote region procedure for {}, serverName={}, code={}," +
+          " seqId={}, proc={}, should be a retry, ignore",
+        regionNode, serverName, code, seqId, this);
       return;
     }
-    if (openSeqNum < 0) {
-      throw new UnexpectedStateException("Received report unexpected " + TransitionCode.OPENED +
-        " transition openSeqNum=" + openSeqNum + ", " + regionNode + ", proc=" + this);
-    }
-    if (openSeqNum < regionNode.getOpenSeqNum()) {
-      // use the openSeqNum as a fence, if this is not a retry, then the openSeqNum should be
-      // greater than or equal to the existing one.
+    // The procId could be -1 if it is from an old region server, we need to deal with it so that we
+    // can do rolling upgraing.
+    if (procId >= 0 && remoteProc.getProcId() != procId) {
       LOG.warn(
-        "Received report {} transition from {} for {}, pid={} but the new openSeqNum {}" +
-          " is less than the current one {}, should be a retry, ignore",
-        TransitionCode.OPENED, serverName, regionNode, getProcId(), openSeqNum,
-        regionNode.getOpenSeqNum());
-      return;
-    }
-    // notice that it is possible for a region to still have the same openSeqNum if it crashes and
-    // we haven't written anything into it. That's why we can not just change the above condition
-    // from '<' to '<='. So here we still need to check whether the serverName
-    // matches, to determine whether this is a retry when the openSeqNum is not changed.
-    if (!regionNode.getRegionLocation().equals(serverName)) {
-      LOG.warn("Received report {} transition from {} for {}, pid={} but the region is not on it," +
-        " should be a retry, ignore", TransitionCode.OPENED, serverName, regionNode, getProcId());
+        "The pid of remote region procedure for {} is {}, the reported pid={}, serverName={}," +
+          " code={}, seqId={}, proc={}, should be a retry, ignore",
+        regionNode, remoteProc.getProcId(), procId, serverName, code, seqId, this);
       return;
     }
-    regionNode.setOpenSeqNum(openSeqNum);
-    env.getAssignmentManager().regionOpened(regionNode);
-    if (lastState == RegionStateTransitionState.REGION_STATE_TRANSITION_CONFIRM_OPENED) {
-      // if parent procedure is ServerCrashProcedure, update progress
-      ServerCrashProcedure.updateProgress(env, getParentProcId());
-      // we are done
-      regionNode.unsetProcedure(this);
-    }
-    regionNode.getProcedureEvent().wake(env.getProcedureScheduler());
+    remoteProc.reportTransition(env, regionNode, serverName, code, seqId);
   }
 
-  private void reportTransitionFailedOpen(MasterProcedureEnv env, RegionStateNode regionNode,
-      ServerName serverName) {
-    if (!isOpening(regionNode, serverName, TransitionCode.FAILED_OPEN)) {
-      return;
-    }
-    // there is no openSeqNum for FAILED_OPEN, so we will check the target server instead
-    if (!regionNode.getRegionLocation().equals(serverName)) {
-      LOG.warn(
-        "Received report {} transition from {} for {}, pid={}," +
-          " but the region is not on it, should be a retry, ignore",
-        TransitionCode.FAILED_OPEN, regionNode, serverName, getProcId());
-      return;
-    }
-    // just wake up the procedure and see if we can retry
-    // Notice that, even if we arrive here, this call could still be a retry, as we may retry
-    // opening on the same server again. And the assumption here is that, once the region state is
-    // OPENING, and the TRSP state is REGION_STATE_TRANSITION_CONFIRM_OPENED, the TRSP must have
-    // been suspended on the procedure event, so after the waking operation here, the TRSP will be
-    // executed and try to schedule new OpenRegionProcedure again. Once there is a successful open
-    // then we are done, so the TRSP will not be stuck.
-    // TODO: maybe we could send the procedure id of the OpenRegionProcedure to the region server
-    // and let the region server send it back when done, so it will be easy to detect whether this
-    // is a retry.
-    regionNode.getProcedureEvent().wake(env.getProcedureScheduler());
-  }
-
-  // we do not need seqId for closing a region
-  private void reportTransitionClosed(MasterProcedureEnv env, RegionStateNode regionNode,
+  // Should be called with RegionStateNode locked
+  public void serverCrashed(MasterProcedureEnv env, RegionStateNode regionNode,
       ServerName serverName) throws IOException {
-    if (!regionNode.isInState(State.CLOSING)) {
-      LOG.warn(
-        "Received report {} transition from {} for {}, pid={}" +
-          ", but the region is not in CLOSING state, should be a retry, ignore",
-        TransitionCode.CLOSED, serverName, regionNode, getProcId());
-      return;
-    }
-    if (getCurrentState() != REGION_STATE_TRANSITION_CONFIRM_CLOSED) {
-      LOG.warn(
-        "Received report {} transition from {} for {}, pid={} but the proc is not in {}" +
-          " state, should be a retry, ignore",
-        TransitionCode.CLOSED, serverName, regionNode, getProcId(),
-        REGION_STATE_TRANSITION_CONFIRM_CLOSED);
-      return;
-    }
-    if (!regionNode.getRegionLocation().equals(serverName)) {
-      LOG.warn(
-        "Received report {} transition from {} for {}, pid={}," +
-          " but the region is not on it, should be a retry, ignore",
-        TransitionCode.CLOSED, serverName, regionNode, getProcId());
-      return;
-    }
-    env.getAssignmentManager().regionClosed(regionNode, true);
-    if (lastState == RegionStateTransitionState.REGION_STATE_TRANSITION_CONFIRM_CLOSED) {
-      // we are done
-      regionNode.unsetProcedure(this);
+    if (remoteProc != null) {
+      // this means we are waiting for the sub procedure, so wake it up
+      remoteProc.serverCrashed(env, regionNode, serverName);
+    } else {
+      // we are in RUNNING state, just update the region state, and we will process it later.
+      env.getAssignmentManager().regionClosed(regionNode, false);
     }
-    regionNode.getProcedureEvent().wake(env.getProcedureScheduler());
   }
 
-  // Should be called with RegionStateNode locked
-  public void reportTransition(MasterProcedureEnv env, RegionStateNode regionNode,
-      ServerName serverName, TransitionCode code, long seqId) throws IOException {
-    // It is possible that the previous reportRegionStateTransition call was succeeded at master
-    // side, but before returning the result to region server, the rpc connection was broken, or the
-    // master restarted. The region server will try calling reportRegionStateTransition again under
-    // this scenario, so here we need to check whether this is a retry.
-    switch (code) {
-      case OPENED:
-        reportTransitionOpen(env, regionNode, serverName, seqId);
-        break;
-      case FAILED_OPEN:
-        reportTransitionFailedOpen(env, regionNode, serverName);
-        break;
-      case CLOSED:
-        reportTransitionClosed(env, regionNode, serverName);
-        break;
-      default:
-        throw new UnexpectedStateException("Received report unexpected " + code + " transition, " +
-          regionNode.toShortString() + ", " + this + ", expected OPENED or FAILED_OPEN or CLOSED.");
-    }
+  void attachRemoteProc(RegionRemoteProcedureBase proc) {
+    this.remoteProc = proc;
   }
 
-  // Should be called with RegionStateNode locked
-  public void serverCrashed(MasterProcedureEnv env, RegionStateNode regionNode,
-      ServerName serverName) throws IOException {
-    // Notice that, in this method, we do not change the procedure state, instead, we update the
-    // region state in hbase:meta. This is because that, the procedure state change will not be
-    // persisted until the region is woken up and finish one step, if we crash before that then the
-    // information will be lost. So here we will update the region state in hbase:meta, and when the
-    // procedure is woken up, it will process the error and jump to the correct procedure state.
-    RegionStateTransitionState currentState = getCurrentState();
-    switch (currentState) {
-      case REGION_STATE_TRANSITION_CLOSE:
-      case REGION_STATE_TRANSITION_CONFIRM_CLOSED:
-      case REGION_STATE_TRANSITION_CONFIRM_OPENED:
-        // for these 3 states, the region may still be online on the crashed server
-        env.getAssignmentManager().regionClosed(regionNode, false);
-        if (currentState != RegionStateTransitionState.REGION_STATE_TRANSITION_CLOSE) {
-          regionNode.getProcedureEvent().wake(env.getProcedureScheduler());
-        }
-        break;
-      default:
-        // If the procedure is in other 2 states, then actually we should not arrive here, as we
-        // know that the region is not online on any server, so we need to do nothing... But anyway
-        // let's add a log here
-        LOG.warn("{} received unexpected server crash call for region {} from {}", this, regionNode,
-          serverName);
-
-    }
+  void unattachRemoteProc(RegionRemoteProcedureBase proc) {
+    assert this.remoteProc == proc;
+    this.remoteProc = null;
   }
 
   private boolean incrementAndCheckMaxAttempts(MasterProcedureEnv env, RegionStateNode regionNode) {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/UnassignProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/UnassignProcedure.java
index def8fd5..6f5c4af 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/UnassignProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/UnassignProcedure.java
@@ -23,7 +23,6 @@ import java.io.IOException;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.exceptions.UnexpectedStateException;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
-import org.apache.hadoop.hbase.master.procedure.RSProcedureDispatcher.RegionCloseOperation;
 import org.apache.hadoop.hbase.procedure2.ProcedureMetrics;
 import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
 import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteOperation;
@@ -129,8 +128,7 @@ public class UnassignProcedure extends RegionTransitionProcedure {
   @Override
   public RemoteOperation remoteCallBuild(final MasterProcedureEnv env,
       final ServerName serverName) {
-    assert serverName.equals(getRegionState(env).getRegionLocation());
-    return new RegionCloseOperation(this, getRegionInfo(), this.destinationServer);
+    return null;
   }
 
   @Override
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
index b8ba7b3..b469cb8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
@@ -399,54 +399,36 @@ public class RSProcedureDispatcher
   }
 
   public static abstract class RegionOperation extends RemoteOperation {
-    private final RegionInfo regionInfo;
+    protected final RegionInfo regionInfo;
+    protected final long procId;
 
-    protected RegionOperation(final RemoteProcedure remoteProcedure,
-        final RegionInfo regionInfo) {
+    protected RegionOperation(RemoteProcedure remoteProcedure, RegionInfo regionInfo, long procId) {
       super(remoteProcedure);
       this.regionInfo = regionInfo;
-    }
-
-    public RegionInfo getRegionInfo() {
-      return this.regionInfo;
+      this.procId = procId;
     }
   }
 
   public static class RegionOpenOperation extends RegionOperation {
-    private final List<ServerName> favoredNodes;
-    private final boolean openForReplay;
-    private boolean failedOpen;
 
-    public RegionOpenOperation(final RemoteProcedure remoteProcedure,
-        final RegionInfo regionInfo, final List<ServerName> favoredNodes,
-        final boolean openForReplay) {
-      super(remoteProcedure, regionInfo);
-      this.favoredNodes = favoredNodes;
-      this.openForReplay = openForReplay;
-    }
-
-    protected void setFailedOpen(final boolean failedOpen) {
-      this.failedOpen = failedOpen;
-    }
-
-    public boolean isFailedOpen() {
-      return failedOpen;
+    public RegionOpenOperation(RemoteProcedure remoteProcedure, RegionInfo regionInfo,
+        long procId) {
+      super(remoteProcedure, regionInfo, procId);
     }
 
     public OpenRegionRequest.RegionOpenInfo buildRegionOpenInfoRequest(
         final MasterProcedureEnv env) {
-      return RequestConverter.buildRegionOpenInfo(getRegionInfo(),
-        env.getAssignmentManager().getFavoredNodes(getRegionInfo()));
+      return RequestConverter.buildRegionOpenInfo(regionInfo,
+        env.getAssignmentManager().getFavoredNodes(regionInfo), procId);
     }
   }
 
   public static class RegionCloseOperation extends RegionOperation {
     private final ServerName destinationServer;
-    private boolean closed = false;
 
-    public RegionCloseOperation(final RemoteProcedure remoteProcedure,
-        final RegionInfo regionInfo, final ServerName destinationServer) {
-      super(remoteProcedure, regionInfo);
+    public RegionCloseOperation(RemoteProcedure remoteProcedure, RegionInfo regionInfo, long procId,
+        ServerName destinationServer) {
+      super(remoteProcedure, regionInfo, procId);
       this.destinationServer = destinationServer;
     }
 
@@ -454,17 +436,9 @@ public class RSProcedureDispatcher
       return destinationServer;
     }
 
-    protected void setClosed(final boolean closed) {
-      this.closed = closed;
-    }
-
-    public boolean isClosed() {
-      return closed;
-    }
-
     public CloseRegionRequest buildCloseRegionRequest(final ServerName serverName) {
-      return ProtobufUtil.buildCloseRegionRequest(serverName,
-        getRegionInfo().getRegionName(), getDestinationServer());
+      return ProtobufUtil.buildCloseRegionRequest(serverName, regionInfo.getRegionName(),
+        getDestinationServer(), procId);
     }
   }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index e407285..bcb1a07 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -2228,9 +2228,11 @@ public class HRegionServer extends HasThread implements
   @Override
   public void postOpenDeployTasks(final PostOpenDeployContext context) throws IOException {
     HRegion r = context.getRegion();
+    long openProcId = context.getOpenProcId();
     long masterSystemTime = context.getMasterSystemTime();
     rpcServices.checkOpen();
-    LOG.info("Post open deploy tasks for " + r.getRegionInfo().getRegionNameAsString());
+    LOG.info("Post open deploy tasks for {}, openProcId={}, masterSystemTime={}",
+      r.getRegionInfo().getRegionNameAsString(), openProcId, masterSystemTime);
     // Do checks to see if we need to compact (references or too many files)
     for (HStore s : r.stores.values()) {
       if (s.hasReferences() || s.needsCompaction()) {
@@ -2247,7 +2249,7 @@ public class HRegionServer extends HasThread implements
 
     // Notify master
     if (!reportRegionStateTransition(new RegionStateTransitionContext(TransitionCode.OPENED,
-      openSeqNum, masterSystemTime, r.getRegionInfo()))) {
+      openSeqNum, openProcId, masterSystemTime, r.getRegionInfo()))) {
       throw new IOException(
         "Failed to report opened region to master: " + r.getRegionInfo().getRegionNameAsString());
     }
@@ -2263,6 +2265,7 @@ public class HRegionServer extends HasThread implements
     long openSeqNum = context.getOpenSeqNum();
     long masterSystemTime = context.getMasterSystemTime();
     RegionInfo[] hris = context.getHris();
+    long[] procIds = context.getProcIds();
 
     if (TEST_SKIP_REPORTING_TRANSITION) {
       // This is for testing only in case there is no master
@@ -2301,6 +2304,9 @@ public class HRegionServer extends HasThread implements
     for (RegionInfo hri: hris) {
       transition.addRegionInfo(ProtobufUtil.toRegionInfo(hri));
     }
+    for (long procId: procIds) {
+      transition.addProcId(procId);
+    }
     ReportRegionStateTransitionRequest request = builder.build();
     int tries = 0;
     long pauseTime = INIT_PAUSE_TIME_MS;
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index 9b99ff8..2054ac7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -3714,8 +3714,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
         regionServer.updateRegionFavoredNodesMapping(regionInfo.getEncodedName(),
           regionOpenInfo.getFavoredNodesList());
       }
-      regionServer.executorService
-        .submit(AssignRegionHandler.create(regionServer, regionInfo, tableDesc, masterSystemTime));
+      regionServer.executorService.submit(AssignRegionHandler.create(regionServer, regionInfo,
+        regionOpenInfo.getOpenProcId(), tableDesc, masterSystemTime));
     }
   }
 
@@ -3729,8 +3729,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     ServerName destination =
       request.hasDestinationServer() ? ProtobufUtil.toServerName(request.getDestinationServer())
         : null;
-    regionServer.executorService
-      .submit(UnassignRegionHandler.create(regionServer, encodedName, false, destination));
+    regionServer.executorService.submit(UnassignRegionHandler.create(regionServer, encodedName,
+      request.getCloseProcId(), false, destination));
   }
 
   private void executeProcedures(RemoteProcedureRequest request) {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java
index e0638ac..17f318b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java
@@ -100,16 +100,23 @@ public interface RegionServerServices extends Server, MutableOnlineRegions, Favo
    */
   class PostOpenDeployContext {
     private final HRegion region;
+    private final long openProcId;
     private final long masterSystemTime;
 
-    @InterfaceAudience.Private
-    public PostOpenDeployContext(HRegion region, long masterSystemTime) {
+    public PostOpenDeployContext(HRegion region, long openProcId, long masterSystemTime) {
       this.region = region;
+      this.openProcId = openProcId;
       this.masterSystemTime = masterSystemTime;
     }
+
     public HRegion getRegion() {
       return region;
     }
+
+    public long getOpenProcId() {
+      return openProcId;
+    }
+
     public long getMasterSystemTime() {
       return masterSystemTime;
     }
@@ -125,28 +132,46 @@ public interface RegionServerServices extends Server, MutableOnlineRegions, Favo
     private final TransitionCode code;
     private final long openSeqNum;
     private final long masterSystemTime;
+    private final long[] procIds;
     private final RegionInfo[] hris;
 
-    @InterfaceAudience.Private
     public RegionStateTransitionContext(TransitionCode code, long openSeqNum, long masterSystemTime,
         RegionInfo... hris) {
       this.code = code;
       this.openSeqNum = openSeqNum;
       this.masterSystemTime = masterSystemTime;
       this.hris = hris;
+      this.procIds = new long[hris.length];
     }
+
+    public RegionStateTransitionContext(TransitionCode code, long openSeqNum, long procId,
+        long masterSystemTime, RegionInfo hri) {
+      this.code = code;
+      this.openSeqNum = openSeqNum;
+      this.masterSystemTime = masterSystemTime;
+      this.hris = new RegionInfo[] { hri };
+      this.procIds = new long[] { procId };
+    }
+
     public TransitionCode getCode() {
       return code;
     }
+
     public long getOpenSeqNum() {
       return openSeqNum;
     }
+
     public long getMasterSystemTime() {
       return masterSystemTime;
     }
+
     public RegionInfo[] getHris() {
       return hris;
     }
+
+    public long[] getProcIds() {
+      return procIds;
+    }
   }
 
   /**
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitRequest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitRequest.java
index 9a0531c..9e806bb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitRequest.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitRequest.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hbase.regionserver;
 
 import java.security.PrivilegedAction;
-
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.RegionInfo;
@@ -30,7 +29,9 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
+
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
 
 /**
@@ -87,7 +88,7 @@ class SplitRequest implements Runnable {
     // hri_a and hri_b objects may not reflect the regions that will be created, those objects
     // are created just to pass the information to the reportRegionStateTransition().
     if (!server.reportRegionStateTransition(new RegionStateTransitionContext(
-        TransitionCode.READY_TO_SPLIT, HConstants.NO_SEQNUM, -1, parent, hri_a, hri_b))) {
+      TransitionCode.READY_TO_SPLIT, HConstants.NO_SEQNUM, -1, parent, hri_a, hri_b))) {
       LOG.error("Unable to ask master to split " + parent.getRegionNameAsString());
     }
   }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/AssignRegionHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/AssignRegionHandler.java
index c6fee2e..bc2425b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/AssignRegionHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/AssignRegionHandler.java
@@ -51,16 +51,19 @@ public class AssignRegionHandler extends EventHandler {
 
   private final RegionInfo regionInfo;
 
+  private final long openProcId;
+
   private final TableDescriptor tableDesc;
 
   private final long masterSystemTime;
 
   private final RetryCounter retryCounter;
 
-  public AssignRegionHandler(RegionServerServices server, RegionInfo regionInfo,
+  public AssignRegionHandler(RegionServerServices server, RegionInfo regionInfo, long openProcId,
       @Nullable TableDescriptor tableDesc, long masterSystemTime, EventType eventType) {
     super(server, eventType);
     this.regionInfo = regionInfo;
+    this.openProcId = openProcId;
     this.tableDesc = tableDesc;
     this.masterSystemTime = masterSystemTime;
     this.retryCounter = HandlerUtil.getRetryCounter();
@@ -76,7 +79,7 @@ public class AssignRegionHandler extends EventHandler {
     RegionServerServices rs = getServer();
     rs.getRegionsInTransitionInRS().remove(regionInfo.getEncodedNameAsBytes(), Boolean.TRUE);
     if (!rs.reportRegionStateTransition(new RegionStateTransitionContext(TransitionCode.FAILED_OPEN,
-      HConstants.NO_SEQNUM, masterSystemTime, regionInfo))) {
+      HConstants.NO_SEQNUM, openProcId, masterSystemTime, regionInfo))) {
       throw new IOException(
         "Failed to report failed open to master: " + regionInfo.getRegionNameAsString());
     }
@@ -133,7 +136,7 @@ public class AssignRegionHandler extends EventHandler {
       cleanUpAndReportFailure(e);
       return;
     }
-    rs.postOpenDeployTasks(new PostOpenDeployContext(region, masterSystemTime));
+    rs.postOpenDeployTasks(new PostOpenDeployContext(region, openProcId, masterSystemTime));
     rs.addRegion(region);
     LOG.info("Opened {}", regionName);
     Boolean current = rs.getRegionsInTransitionInRS().remove(regionInfo.getEncodedNameAsBytes());
@@ -156,7 +159,7 @@ public class AssignRegionHandler extends EventHandler {
   }
 
   public static AssignRegionHandler create(RegionServerServices server, RegionInfo regionInfo,
-      TableDescriptor tableDesc, long masterSystemTime) {
+      long openProcId, TableDescriptor tableDesc, long masterSystemTime) {
     EventType eventType;
     if (regionInfo.isMetaRegion()) {
       eventType = EventType.M_RS_CLOSE_META;
@@ -166,6 +169,7 @@ public class AssignRegionHandler extends EventHandler {
     } else {
       eventType = EventType.M_RS_OPEN_REGION;
     }
-    return new AssignRegionHandler(server, regionInfo, tableDesc, masterSystemTime, eventType);
+    return new AssignRegionHandler(server, regionInfo, openProcId, tableDesc, masterSystemTime,
+      eventType);
   }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java
index 0e35a0b..d4ea004 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.executor.EventHandler;
 import org.apache.hadoop.hbase.executor.EventType;
+import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices.RegionStateTransitionContext;
@@ -123,7 +124,7 @@ public class CloseRegionHandler extends EventHandler {
 
       this.rsServices.removeRegion(region, destination);
       rsServices.reportRegionStateTransition(new RegionStateTransitionContext(TransitionCode.CLOSED,
-          HConstants.NO_SEQNUM, -1, regionInfo));
+        HConstants.NO_SEQNUM, Procedure.NO_PROC_ID, -1, regionInfo));
 
       // Done!  Region is closed on this RS
       LOG.debug("Closed " + region.getRegionInfo().getRegionNameAsString());
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java
index 31177ef..8b644b0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.executor.EventHandler;
 import org.apache.hadoop.hbase.executor.EventType;
+import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
@@ -156,15 +157,14 @@ public class OpenRegionHandler extends EventHandler {
     }
   }
 
-  private void doCleanUpOnFailedOpen(HRegion region)
-      throws IOException {
+  private void doCleanUpOnFailedOpen(HRegion region) throws IOException {
     try {
       if (region != null) {
         cleanupFailedOpen(region);
       }
     } finally {
       rsServices.reportRegionStateTransition(new RegionStateTransitionContext(
-          TransitionCode.FAILED_OPEN, HConstants.NO_SEQNUM, -1, regionInfo));
+        TransitionCode.FAILED_OPEN, HConstants.NO_SEQNUM, Procedure.NO_PROC_ID, -1, regionInfo));
     }
   }
 
@@ -248,19 +248,19 @@ public class OpenRegionHandler extends EventHandler {
     @Override
     public void run() {
       try {
-        this.services.postOpenDeployTasks(new PostOpenDeployContext(region, masterSystemTime));
+        this.services.postOpenDeployTasks(
+          new PostOpenDeployContext(region, Procedure.NO_PROC_ID, masterSystemTime));
       } catch (Throwable e) {
         String msg = "Exception running postOpenDeployTasks; region=" +
           this.region.getRegionInfo().getEncodedName();
         this.exception = e;
-        if (e instanceof IOException
-            && isRegionStillOpening(region.getRegionInfo(), services)) {
+        if (e instanceof IOException && isRegionStillOpening(region.getRegionInfo(), services)) {
           server.abort(msg, e);
         } else {
           LOG.warn(msg, e);
         }
       }
-      // We're done.  Set flag then wake up anyone waiting on thread to complete.
+      // We're done. Set flag then wake up anyone waiting on thread to complete.
       this.signaller.set(true);
       synchronized (this.signaller) {
         this.signaller.notify();
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/UnassignRegionHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/UnassignRegionHandler.java
index cd38db1..3ce7caa 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/UnassignRegionHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/UnassignRegionHandler.java
@@ -49,18 +49,22 @@ public class UnassignRegionHandler extends EventHandler {
   private static final Logger LOG = LoggerFactory.getLogger(UnassignRegionHandler.class);
 
   private final String encodedName;
+
+  private final long closeProcId;
   // If true, the hosting server is aborting. Region close process is different
   // when we are aborting.
+  // TODO: not used yet, we still use the old CloseRegionHandler when aborting
   private final boolean abort;
 
   private final ServerName destination;
 
   private final RetryCounter retryCounter;
 
-  public UnassignRegionHandler(RegionServerServices server, String encodedName, boolean abort,
-      @Nullable ServerName destination, EventType eventType) {
+  public UnassignRegionHandler(RegionServerServices server, String encodedName, long closeProcId,
+      boolean abort, @Nullable ServerName destination, EventType eventType) {
     super(server, eventType);
     this.encodedName = encodedName;
+    this.closeProcId = closeProcId;
     this.abort = abort;
     this.destination = destination;
     this.retryCounter = HandlerUtil.getRetryCounter();
@@ -117,7 +121,7 @@ public class UnassignRegionHandler extends EventHandler {
     }
     rs.removeRegion(region, destination);
     if (!rs.reportRegionStateTransition(new RegionStateTransitionContext(TransitionCode.CLOSED,
-      HConstants.NO_SEQNUM, -1, region.getRegionInfo()))) {
+      HConstants.NO_SEQNUM, closeProcId, -1, region.getRegionInfo()))) {
       throw new IOException("Failed to report close to master: " + regionName);
     }
     rs.getRegionsInTransitionInRS().remove(encodedNameBytes, Boolean.FALSE);
@@ -131,13 +135,14 @@ public class UnassignRegionHandler extends EventHandler {
   }
 
   public static UnassignRegionHandler create(RegionServerServices server, String encodedName,
-      boolean abort, @Nullable ServerName destination) {
+      long closeProcId, boolean abort, @Nullable ServerName destination) {
     // Just try our best to determine whether it is for closing meta. It is not the end of the world
     // if we put the handler into a wrong executor.
     Region region = server.getRegion(encodedName);
     EventType eventType =
       region != null && region.getRegionInfo().isMetaRegion() ? EventType.M_RS_CLOSE_META
         : EventType.M_RS_CLOSE_REGION;
-    return new UnassignRegionHandler(server, encodedName, abort, destination, eventType);
+    return new UnassignRegionHandler(server, encodedName, closeProcId, abort, destination,
+      eventType);
   }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManagerBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManagerBase.java
index fb6668a..9f3aceb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManagerBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManagerBase.java
@@ -400,6 +400,13 @@ public abstract class TestAssignmentManagerBase {
         if (retries == timeoutTimes) {
           LOG.info("Mark server=" + server + " as dead. retries=" + retries);
           master.getServerManager().moveFromOnlineToDeadServers(server);
+          executor.schedule(new Runnable() {
+            @Override
+            public void run() {
+              LOG.info("Sending in CRASH of " + server);
+              doCrash(server);
+            }
+          }, 1, TimeUnit.SECONDS);
         }
         throw new SocketTimeoutException("simulate socket timeout");
       } else {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestCloseRegionWhileRSCrash.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestCloseRegionWhileRSCrash.java
index d34bfbb..0a29958 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestCloseRegionWhileRSCrash.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestCloseRegionWhileRSCrash.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.ServerManager;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.master.procedure.ServerCrashProcedure;
@@ -167,7 +168,7 @@ public class TestCloseRegionWhileRSCrash {
     HRegionServer dstRs = UTIL.getOtherRegionServer(srcRs);
     ProcedureExecutor<MasterProcedureEnv> procExec =
       UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor();
-    long dummyProcId = procExec.submitProcedure(new DummyServerProcedure(srcRs.getServerName()));
+    procExec.submitProcedure(new DummyServerProcedure(srcRs.getServerName()));
     ARRIVE.await();
     UTIL.getMiniHBaseCluster().killRegionServer(srcRs.getServerName());
     UTIL.waitFor(30000,
@@ -185,13 +186,12 @@ public class TestCloseRegionWhileRSCrash {
       30000);
     // wait until the timeout value increase three times
     ProcedureTestUtil.waitUntilProcedureTimeoutIncrease(UTIL, TransitRegionStateProcedure.class, 3);
-    // let's close the connection to make sure that the SCP can not update meta successfully
-    UTIL.getMiniHBaseCluster().getMaster().getConnection().close();
+    // close connection to make sure that we can not finish the TRSP
+    HMaster master = UTIL.getMiniHBaseCluster().getMaster();
+    master.getConnection().close();
     RESUME.countDown();
-    UTIL.waitFor(30000, () -> procExec.isFinished(dummyProcId));
-    Thread.sleep(2000);
-    // here we restart
-    UTIL.getMiniHBaseCluster().stopMaster(0).join();
+    UTIL.waitFor(30000, () -> !master.isAlive());
+    // here we start a new master
     UTIL.getMiniHBaseCluster().startMaster();
     t.join();
     // Make sure that the region is online, it may not on the original target server, as we will set
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureHang.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureHang.java
new file mode 100644
index 0000000..0463721
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureHang.java
@@ -0,0 +1,209 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.assignment;
+
+import java.io.IOException;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.PleaseHoldException;
+import org.apache.hadoop.hbase.StartMiniClusterOption;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.hadoop.hbase.master.RegionPlan;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
+import org.apache.zookeeper.KeeperException;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse;
+
+/**
+ * See HBASE-22060 and HBASE-22074 for more details.
+ */
+@Category({ MasterTests.class, MediumTests.class })
+public class TestOpenRegionProcedureHang {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestOpenRegionProcedureHang.class);
+
+  private static final Logger LOG = LoggerFactory.getLogger(TestOpenRegionProcedureHang.class);
+
+  private static CountDownLatch ARRIVE;
+  private static CountDownLatch RESUME;
+
+  private static CountDownLatch FINISH;
+
+  private static CountDownLatch ABORT;
+
+  private static final class AssignmentManagerForTest extends AssignmentManager {
+
+    public AssignmentManagerForTest(MasterServices master) {
+      super(master);
+    }
+
+    @Override
+    public ReportRegionStateTransitionResponse reportRegionStateTransition(
+        ReportRegionStateTransitionRequest req) throws PleaseHoldException {
+      RegionStateTransition transition = req.getTransition(0);
+      if (transition.getTransitionCode() == TransitionCode.OPENED &&
+        ProtobufUtil.toTableName(transition.getRegionInfo(0).getTableName()).equals(NAME) &&
+        ARRIVE != null) {
+        ARRIVE.countDown();
+        try {
+          RESUME.await();
+          RESUME = null;
+        } catch (InterruptedException e) {
+          throw new RuntimeException(e);
+        }
+        try {
+          return super.reportRegionStateTransition(req);
+        } finally {
+          FINISH.countDown();
+        }
+      } else {
+        return super.reportRegionStateTransition(req);
+      }
+    }
+  }
+
+  public static final class HMasterForTest extends HMaster {
+
+    public HMasterForTest(Configuration conf) throws IOException, KeeperException {
+      super(conf);
+    }
+
+    @Override
+    protected AssignmentManager createAssignmentManager(MasterServices master) {
+      return new AssignmentManagerForTest(master);
+    }
+
+    @Override
+    public void abort(String reason, Throwable cause) {
+      // hang here so we can finish the reportRegionStateTransition call, which is the most
+      // important part to reproduce the bug
+      if (ABORT != null) {
+        try {
+          ABORT.await();
+          ABORT = null;
+        } catch (InterruptedException e) {
+          throw new RuntimeException(e);
+        }
+      }
+      super.abort(reason, cause);
+    }
+  }
+
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  private static TableName NAME = TableName.valueOf("Open");
+
+  private static byte[] CF = Bytes.toBytes("cf");
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    Configuration conf = UTIL.getConfiguration();
+    conf.setClass(HConstants.MASTER_IMPL, HMasterForTest.class, HMaster.class);
+
+    // make sure we do not timeout when caling reportRegionStateTransition
+    conf.setInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT, 10 * 60 * 1000);
+    conf.setInt(HConstants.HBASE_RPC_SHORTOPERATION_TIMEOUT_KEY, 10 * 60 * 1000);
+    UTIL
+      .startMiniCluster(StartMiniClusterOption.builder().numMasters(2).numRegionServers(3).build());
+    UTIL.createTable(NAME, CF);
+    UTIL.waitTableAvailable(NAME);
+    UTIL.getAdmin().balancerSwitch(false, true);
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    UTIL.shutdownMiniCluster();
+  }
+
+  @Test
+  public void test() throws InterruptedException, KeeperException, IOException {
+    RegionInfo region = UTIL.getMiniHBaseCluster().getRegions(NAME).get(0).getRegionInfo();
+    AssignmentManager am = UTIL.getMiniHBaseCluster().getMaster().getAssignmentManager();
+
+    HRegionServer rs1 = UTIL.getRSForFirstRegionInTable(NAME);
+    HRegionServer rs2 = UTIL.getOtherRegionServer(rs1);
+
+    ARRIVE = new CountDownLatch(1);
+    RESUME = new CountDownLatch(1);
+    FINISH = new CountDownLatch(1);
+    ABORT = new CountDownLatch(1);
+    am.moveAsync(new RegionPlan(region, rs1.getServerName(), rs2.getServerName()));
+
+    ARRIVE.await();
+    ARRIVE = null;
+    HMaster master = UTIL.getMiniHBaseCluster().getMaster();
+    master.getZooKeeper().close();
+    UTIL.waitFor(30000, () -> {
+      for (MasterThread mt : UTIL.getMiniHBaseCluster().getMasterThreads()) {
+        if (mt.getMaster() != master && mt.getMaster().isActiveMaster()) {
+          return mt.getMaster().isInitialized();
+        }
+      }
+      return false;
+    });
+    ProcedureExecutor<MasterProcedureEnv> procExec =
+      UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor();
+    UTIL.waitFor(30000,
+      () -> procExec.getProcedures().stream().filter(p -> p instanceof OpenRegionProcedure)
+        .map(p -> (OpenRegionProcedure) p).anyMatch(p -> p.region.getTable().equals(NAME)));
+    OpenRegionProcedure proc = procExec.getProcedures().stream()
+      .filter(p -> p instanceof OpenRegionProcedure).map(p -> (OpenRegionProcedure) p)
+      .filter(p -> p.region.getTable().equals(NAME)).findFirst().get();
+    // wait a bit to let the OpenRegionProcedure send out the request
+    Thread.sleep(2000);
+    RESUME.countDown();
+    if (!FINISH.await(15, TimeUnit.SECONDS)) {
+      LOG.info("Wait reportRegionStateTransition to finish timed out, this is possible if" +
+        " we update the procedure store, as the WALProcedureStore" +
+        " will retry forever to roll the writer if it is not closed");
+    }
+    FINISH = null;
+    // if the reportRegionTransition is finished, wait a bit to let it return the data to RS
+    Thread.sleep(2000);
+    ABORT.countDown();
+
+    UTIL.waitFor(30000, () -> procExec.isFinished(proc.getProcId()));
+    UTIL.waitFor(30000, () -> procExec.isFinished(proc.getParentProcId()));
+  }
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestServerRemoteProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestServerRemoteProcedure.java
index d4745b9..f03794a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestServerRemoteProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestServerRemoteProcedure.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hbase.master.procedure;
 
 import static org.apache.hadoop.hbase.master.procedure.ServerProcedureInterface.ServerOperationType.SWITCH_RPC_THROTTLE;
+import static org.junit.Assert.fail;
 
 import java.io.IOException;
 import java.util.NavigableMap;
@@ -31,7 +32,6 @@ import java.util.concurrent.Future;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
-
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.ServerName;
@@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 import org.apache.hadoop.hbase.master.assignment.MockMasterServices;
 import org.apache.hadoop.hbase.master.assignment.OpenRegionProcedure;
+import org.apache.hadoop.hbase.master.assignment.TransitRegionStateProcedure;
 import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
 import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher;
@@ -62,6 +63,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
+
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
 
 @Category({ MasterTests.class, MediumTests.class })
@@ -134,19 +136,21 @@ public class TestServerRemoteProcedure {
   }
 
   @Test
-  public void testRegionOpenProcedureIsNotHandledByDisPatcher() throws Exception {
+  public void testRegionOpenProcedureIsNotHandledByDispatcher() throws Exception {
     TableName tableName = TableName.valueOf("testRegionOpenProcedureIsNotHandledByDisPatcher");
     RegionInfo hri = RegionInfoBuilder.newBuilder(tableName).setStartKey(Bytes.toBytes(1))
-        .setEndKey(Bytes.toBytes(2)).setSplit(false).setRegionId(0).build();
-    master.getMasterProcedureExecutor().getEnvironment().getAssignmentManager().getRegionStates()
-        .getOrCreateRegionStateNode(hri);
+      .setEndKey(Bytes.toBytes(2)).setSplit(false).setRegionId(0).build();
+    MasterProcedureEnv env = master.getMasterProcedureExecutor().getEnvironment();
+    env.getAssignmentManager().getRegionStates().getOrCreateRegionStateNode(hri);
+    TransitRegionStateProcedure proc = TransitRegionStateProcedure.assign(env, hri, null);
     ServerName worker = master.getServerManager().getOnlineServersList().get(0);
-    OpenRegionProcedure openRegionProcedure = new OpenRegionProcedure(hri, worker);
+    OpenRegionProcedure openRegionProcedure = new OpenRegionProcedure(proc, hri, worker);
     Future<byte[]> future = submitProcedure(openRegionProcedure);
     Thread.sleep(2000);
     rsDispatcher.removeNode(worker);
     try {
       future.get(2000, TimeUnit.MILLISECONDS);
+      fail();
     } catch (TimeoutException e) {
       LOG.info("timeout is expected");
     }


[hbase] 21/49: HBASE-22051 Expect values are hard-coded in the verifications of TestRSGroupsBasics

Posted by op...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

openinx pushed a commit to branch HBASE-21879
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit dac7e802e350fc2bf22957952f8c40ab92ed769b
Author: Xiang LI <li...@freewheel.tv>
AuthorDate: Sat Mar 16 11:56:13 2019 +0800

    HBASE-22051 Expect values are hard-coded in the verifications of TestRSGroupsBasics
    
    Signed-off-by: Xu Cang <xu...@apache.org>
---
 .../hadoop/hbase/rsgroup/TestRSGroupsBasics.java   | 58 +++++++++++++++-------
 1 file changed, 41 insertions(+), 17 deletions(-)

diff --git a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBasics.java b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBasics.java
index 2d74834..e3cb54e 100644
--- a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBasics.java
+++ b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBasics.java
@@ -85,10 +85,10 @@ public class TestRSGroupsBasics extends TestRSGroupsBase {
   @Test
   public void testBasicStartUp() throws IOException {
     RSGroupInfo defaultInfo = rsGroupAdmin.getRSGroupInfo(RSGroupInfo.DEFAULT_GROUP);
-    assertEquals(4, defaultInfo.getServers().size());
-    // Assignment of root and meta regions.
+    assertEquals(NUM_SLAVES_BASE, defaultInfo.getServers().size());
+    // Assignment of meta and rsgroup regions.
     int count = master.getAssignmentManager().getRegionStates().getRegionAssignments().size();
-    // 2 meta, group
+    // 2 (meta and rsgroup)
     assertEquals(2, count);
   }
 
@@ -212,17 +212,25 @@ public class TestRSGroupsBasics extends TestRSGroupsBase {
   @Test
   public void testClearDeadServers() throws Exception {
     LOG.info("testClearDeadServers");
-    final RSGroupInfo newGroup = addGroup(getGroupName(name.getMethodName()), 3);
+
+    // move region servers from default group to new group
+    final int serverCountToMoveToNewGroup = 3;
+    final RSGroupInfo newGroup =
+        addGroup(getGroupName(name.getMethodName()), serverCountToMoveToNewGroup);
+
+    // get the existing dead servers
     NUM_DEAD_SERVERS = cluster.getClusterMetrics().getDeadServerNames().size();
 
-    ServerName targetServer = getServerName(newGroup.getServers().iterator().next());
+    // stop 1 region server in new group
+    ServerName serverToStop = getServerName(newGroup.getServers().iterator().next());
     try {
       // stopping may cause an exception
       // due to the connection loss
-      admin.stopRegionServer(targetServer.getAddress().toString());
+      admin.stopRegionServer(serverToStop.getAddress().toString());
       NUM_DEAD_SERVERS++;
     } catch (Exception e) {
     }
+
     // wait for stopped regionserver to dead server list
     TEST_UTIL.waitFor(WAIT_TIMEOUT, new Waiter.Predicate<Exception>() {
       @Override
@@ -231,29 +239,38 @@ public class TestRSGroupsBasics extends TestRSGroupsBase {
           !master.getServerManager().areDeadServersInProgress();
       }
     });
-    assertFalse(cluster.getClusterMetrics().getLiveServerMetrics().containsKey(targetServer));
-    assertTrue(cluster.getClusterMetrics().getDeadServerNames().contains(targetServer));
-    assertTrue(newGroup.getServers().contains(targetServer.getAddress()));
+    assertFalse(cluster.getClusterMetrics().getLiveServerMetrics().containsKey(serverToStop));
+    assertTrue(cluster.getClusterMetrics().getDeadServerNames().contains(serverToStop));
+    assertTrue(newGroup.getServers().contains(serverToStop.getAddress()));
 
     // clear dead servers list
-    List<ServerName> notClearedServers = admin.clearDeadServers(Lists.newArrayList(targetServer));
+    List<ServerName> notClearedServers = admin.clearDeadServers(Lists.newArrayList(serverToStop));
     assertEquals(0, notClearedServers.size());
 
+    // the stopped region server gets cleared and removed from the group
     Set<Address> newGroupServers = rsGroupAdmin.getRSGroupInfo(newGroup.getName()).getServers();
-    assertFalse(newGroupServers.contains(targetServer.getAddress()));
-    assertEquals(2, newGroupServers.size());
+    assertFalse(newGroupServers.contains(serverToStop.getAddress()));
+    assertEquals(serverCountToMoveToNewGroup - 1 /* 1 stopped */, newGroupServers.size());
   }
 
   @Test
   public void testClearNotProcessedDeadServer() throws Exception {
     LOG.info("testClearNotProcessedDeadServer");
+
+    // get the existing dead servers
     NUM_DEAD_SERVERS = cluster.getClusterMetrics().getDeadServerNames().size();
-    RSGroupInfo appInfo = addGroup("deadServerGroup", 1);
-    ServerName targetServer = getServerName(appInfo.getServers().iterator().next());
+
+    // move region servers from default group to "dead server" group
+    final int serverCountToMoveToDeadServerGroup = 1;
+    RSGroupInfo deadServerGroup =
+        addGroup("deadServerGroup", serverCountToMoveToDeadServerGroup);
+
+    // stop 1 region servers in "dead server" group
+    ServerName serverToStop = getServerName(deadServerGroup.getServers().iterator().next());
     try {
       // stopping may cause an exception
       // due to the connection loss
-      admin.stopRegionServer(targetServer.getAddress().toString());
+      admin.stopRegionServer(serverToStop.getAddress().toString());
       NUM_DEAD_SERVERS++;
     } catch (Exception e) {
     }
@@ -263,8 +280,15 @@ public class TestRSGroupsBasics extends TestRSGroupsBase {
         return cluster.getClusterMetrics().getDeadServerNames().size() == NUM_DEAD_SERVERS;
       }
     });
-    List<ServerName> notClearedServers = admin.clearDeadServers(Lists.newArrayList(targetServer));
-    assertEquals(1, notClearedServers.size());
+
+    // the one and only region server in the group does not get cleared, even though it is stopped
+    List<ServerName> notClearedServers = admin.clearDeadServers(Lists.newArrayList(serverToStop));
+    assertEquals(serverCountToMoveToDeadServerGroup, notClearedServers.size());
+
+    Set<Address> ServersInDeadServerGroup =
+        rsGroupAdmin.getRSGroupInfo(deadServerGroup.getName()).getServers();
+    assertEquals(serverCountToMoveToDeadServerGroup, ServersInDeadServerGroup.size());
+    assertTrue(ServersInDeadServerGroup.contains(serverToStop.getAddress()));
   }
 
   @Test


[hbase] 48/49: HBASE-22057 Cap the size of the nodes we delete in one ZK.multi call

Posted by op...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

openinx pushed a commit to branch HBASE-21879
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit a353a8880d159249e3d4ed9f22c0dc892f97bca5
Author: Josh Elser <el...@apache.org>
AuthorDate: Mon Mar 18 16:14:08 2019 -0400

    HBASE-22057 Cap the size of the nodes we delete in one ZK.multi call
    
    If we try to delete too many znodes at once, we'll smack into the
    jute.maxbuffer size. Try to prevent that from happening. The dominating
    factor of the ZK client request should be the znode side on a delete.
    
    Signed-off-by: Peter Somogyi <ps...@apache.org>
---
 .../hbase/zookeeper/RecoverableZooKeeper.java      | 23 ++++--
 .../org/apache/hadoop/hbase/zookeeper/ZKUtil.java  | 69 ++++++++++++++--
 .../apache/hadoop/hbase/zookeeper/TestZKMulti.java | 95 +++++++++++++++++++---
 3 files changed, 159 insertions(+), 28 deletions(-)

diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java
index c23e3d2..757889b 100644
--- a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java
@@ -82,18 +82,13 @@ public class RecoverableZooKeeper {
   private Watcher watcher;
   private int sessionTimeout;
   private String quorumServers;
-
-  public RecoverableZooKeeper(String quorumServers, int sessionTimeout,
-      Watcher watcher, int maxRetries, int retryIntervalMillis, int maxSleepTime)
-    throws IOException {
-    this(quorumServers, sessionTimeout, watcher, maxRetries, retryIntervalMillis, maxSleepTime,
-        null);
-  }
+  private int maxMultiSize;
 
   @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="DE_MIGHT_IGNORE",
       justification="None. Its always been this way.")
   public RecoverableZooKeeper(String quorumServers, int sessionTimeout,
-      Watcher watcher, int maxRetries, int retryIntervalMillis, int maxSleepTime, String identifier)
+      Watcher watcher, int maxRetries, int retryIntervalMillis, int maxSleepTime, String identifier,
+      int maxMultiSize)
     throws IOException {
     // TODO: Add support for zk 'chroot'; we don't add it to the quorumServers String as we should.
     this.retryCounterFactory =
@@ -111,6 +106,7 @@ public class RecoverableZooKeeper {
     this.watcher = watcher;
     this.sessionTimeout = sessionTimeout;
     this.quorumServers = quorumServers;
+    this.maxMultiSize = maxMultiSize;
 
     try {
       checkZk();
@@ -120,6 +116,17 @@ public class RecoverableZooKeeper {
   }
 
   /**
+   * Returns the maximum size (in bytes) that should be included in any single multi() call.
+   *
+   * NB: This is an approximation, so there may be variance in the msg actually sent over the
+   * wire. Please be sure to set this approximately, with respect to your ZK server configuration
+   * for jute.maxbuffer.
+   */
+  public int getMaxMultiSizeLimit() {
+    return maxMultiSize;
+  }
+
+  /**
    * Try to create a ZooKeeper connection. Turns any exception encountered into a
    * KeeperException.OperationTimeoutException so it can retried.
    * @return The created ZooKeeper connection object
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
index 57c847c..51401b0 100644
--- a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
@@ -31,11 +31,14 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Deque;
 import java.util.HashMap;
+import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
+
 import javax.security.auth.login.AppConfigurationEntry;
 import javax.security.auth.login.AppConfigurationEntry.LoginModuleControlFlag;
+
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.AuthUtil;
@@ -73,7 +76,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
-
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
 
@@ -135,8 +137,9 @@ public final class ZKUtil {
     int maxSleepTime = conf.getInt("zookeeper.recovery.retry.maxsleeptime", 60000);
     zkDumpConnectionTimeOut = conf.getInt("zookeeper.dump.connection.timeout",
         1000);
+    int multiMaxSize = conf.getInt("zookeeper.multi.max.size", 1024*1024);
     return new RecoverableZooKeeper(ensemble, timeout, watcher,
-        retry, retryIntervalMillis, maxSleepTime, identifier);
+        retry, retryIntervalMillis, maxSleepTime, identifier, multiMaxSize);
   }
 
   /**
@@ -1333,10 +1336,7 @@ public final class ZKUtil {
         ops.add(ZKUtilOp.deleteNodeFailSilent(children.get(i)));
       }
     }
-    // atleast one element should exist
-    if (ops.size() > 0) {
-      multiOrSequential(zkw, ops, runSequentialOnMultiFailure);
-    }
+    submitBatchedMultiOrSequential(zkw, runSequentialOnMultiFailure, ops);
   }
 
   /**
@@ -1392,13 +1392,64 @@ public final class ZKUtil {
         zkw.interruptedException(e);
       }
     }
-    // atleast one element should exist
-    if (ops.size() > 0) {
-      multiOrSequential(zkw, ops, runSequentialOnMultiFailure);
+    submitBatchedMultiOrSequential(zkw, runSequentialOnMultiFailure, ops);
+  }
+
+  /**
+   * Chunks the provided {@code ops} when their approximate size exceeds the the configured limit.
+   * Take caution that this can ONLY be used for operations where atomicity is not important,
+   * e.g. deletions. It must not be used when atomicity of the operations is critical.
+   */
+  static void submitBatchedMultiOrSequential(ZKWatcher zkw, boolean runSequentialOnMultiFailure,
+      List<ZKUtilOp> ops) throws KeeperException {
+    // at least one element should exist
+    if (ops.isEmpty()) {
+      return;
+    }
+    final int maxMultiSize = zkw.getRecoverableZooKeeper().getMaxMultiSizeLimit();
+    // Batch up the items to over smashing through jute.maxbuffer with too many Ops.
+    final List<List<ZKUtilOp>> batchedOps = partitionOps(ops, maxMultiSize);
+    // Would use forEach() but have to handle KeeperException
+    for (List<ZKUtilOp> batch : batchedOps) {
+      multiOrSequential(zkw, batch, runSequentialOnMultiFailure);
     }
   }
 
   /**
+   * Partition the list of {@code ops} by size (using {@link #estimateSize(ZKUtilOp)}).
+   */
+  static List<List<ZKUtilOp>> partitionOps(List<ZKUtilOp> ops, int maxPartitionSize) {
+    List<List<ZKUtilOp>> partitionedOps = new ArrayList<>();
+    List<ZKUtilOp> currentPartition = new ArrayList<>();
+    int currentPartitionSize = 0;
+    partitionedOps.add(currentPartition);
+    Iterator<ZKUtilOp> iter = ops.iterator();
+    while (iter.hasNext()) {
+      ZKUtilOp currentOp = iter.next();
+      int currentOpSize = estimateSize(currentOp);
+
+      // Roll a new partition if necessary
+      // If the current partition is empty, put the element in there anyways.
+      // We can roll a new partition if we get another element
+      if (!currentPartition.isEmpty() && currentOpSize + currentPartitionSize > maxPartitionSize) {
+        currentPartition = new ArrayList<>();
+        partitionedOps.add(currentPartition);
+        currentPartitionSize = 0;
+      }
+
+      // Add the current op to the partition
+      currentPartition.add(currentOp);
+      // And record its size
+      currentPartitionSize += currentOpSize;
+    }
+    return partitionedOps;
+  }
+
+  static int estimateSize(ZKUtilOp op) {
+    return Bytes.toBytes(op.getPath()).length;
+  }
+
+  /**
    * BFS Traversal of all the children under path, with the entries in the list,
    * in the same order as that of the traversal. Lists all the children without
    * setting any watches.
diff --git a/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKMulti.java b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKMulti.java
index 0f2472b..5508ac7 100644
--- a/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKMulti.java
+++ b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKMulti.java
@@ -17,12 +17,16 @@
  */
 package org.apache.hadoop.hbase.zookeeper;
 
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
 import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
 import java.util.LinkedList;
 import java.util.List;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
@@ -57,21 +61,23 @@ public class TestZKMulti {
   private final static HBaseZKTestingUtility TEST_UTIL = new HBaseZKTestingUtility();
   private static ZKWatcher zkw = null;
 
+  private static class ZKMultiAbortable implements Abortable {
+    @Override
+    public void abort(String why, Throwable e) {
+      LOG.info(why, e);
+    }
+
+    @Override
+    public boolean isAborted() {
+      return false;
+    }
+  }
+
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     TEST_UTIL.startMiniZKCluster();
     Configuration conf = TEST_UTIL.getConfiguration();
-    Abortable abortable = new Abortable() {
-      @Override
-      public void abort(String why, Throwable e) {
-        LOG.info(why, e);
-      }
-
-      @Override
-      public boolean isAborted() {
-        return false;
-      }
-    };
+    Abortable abortable = new ZKMultiAbortable();
     zkw = new ZKWatcher(conf,
       "TestZKMulti", abortable, true);
   }
@@ -368,6 +374,73 @@ public class TestZKMulti {
     assertTrue("Failed to delete child znodes of parent znode 1!", 0 == children.size());
   }
 
+  @Test
+  public void testBatchedDeletesOfWideZNodes() throws Exception {
+    // Batch every 50bytes
+    final int batchSize = 50;
+    Configuration localConf = new Configuration(TEST_UTIL.getConfiguration());
+    localConf.setInt("zookeeper.multi.max.size", batchSize);
+    try (ZKWatcher customZkw = new ZKWatcher(localConf,
+      "TestZKMulti_Custom", new ZKMultiAbortable(), true)) {
+
+      // With a parent znode like this, we'll get batches of 2-3 elements
+      final String parent1 = "/batchedDeletes1";
+      final String parent2 = "/batchedDeletes2";
+      final byte[] EMPTY_BYTES = new byte[0];
+
+      // Write one node
+      List<Op> ops = new ArrayList<>();
+      ops.add(Op.create(parent1, EMPTY_BYTES, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT));
+      for (int i = 0; i < batchSize * 2; i++) {
+        ops.add(Op.create(
+            parent1 + "/" + i, EMPTY_BYTES, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT));
+      }
+      customZkw.getRecoverableZooKeeper().multi(ops);
+
+      // Write into a second node
+      ops.clear();
+      ops.add(Op.create(parent2, EMPTY_BYTES, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT));
+      for (int i = 0; i < batchSize * 4; i++) {
+        ops.add(Op.create(
+            parent2 + "/" + i, EMPTY_BYTES, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT));
+      }
+      customZkw.getRecoverableZooKeeper().multi(ops);
+
+      // These should return successfully
+      ZKUtil.deleteChildrenRecursively(customZkw, parent1);
+      ZKUtil.deleteChildrenRecursively(customZkw, parent2);
+    }
+  }
+
+  @Test
+  public void testListPartitioning() {
+    // 10 Bytes
+    ZKUtilOp tenByteOp = ZKUtilOp.deleteNodeFailSilent("/123456789");
+
+    // Simple, single element case
+    assertEquals(Collections.singletonList(Collections.singletonList(tenByteOp)),
+        ZKUtil.partitionOps(Collections.singletonList(tenByteOp), 15));
+
+    // Simple case where we exceed the limit, but must make the list
+    assertEquals(Collections.singletonList(Collections.singletonList(tenByteOp)),
+        ZKUtil.partitionOps(Collections.singletonList(tenByteOp), 5));
+
+    // Each gets its own bucket
+    assertEquals(
+        Arrays.asList(Arrays.asList(tenByteOp), Arrays.asList(tenByteOp), Arrays.asList(tenByteOp)),
+        ZKUtil.partitionOps(Arrays.asList(tenByteOp, tenByteOp, tenByteOp), 15));
+
+    // Test internal boundary
+    assertEquals(
+        Arrays.asList(Arrays.asList(tenByteOp,tenByteOp), Arrays.asList(tenByteOp)),
+        ZKUtil.partitionOps(Arrays.asList(tenByteOp, tenByteOp, tenByteOp), 20));
+
+    // Plenty of space for one partition
+    assertEquals(
+        Arrays.asList(Arrays.asList(tenByteOp, tenByteOp, tenByteOp)),
+        ZKUtil.partitionOps(Arrays.asList(tenByteOp, tenByteOp, tenByteOp), 50));
+  }
+
   private void createZNodeTree(String rootZNode) throws KeeperException,
       InterruptedException {
     List<Op> opList = new ArrayList<>();


[hbase] 06/49: HBASE-22002 Remove the deprecated methods in Admin interface

Posted by op...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

openinx pushed a commit to branch HBASE-21879
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit 05abf940e823d67a0af3dd7ed9a72508fbbcbc38
Author: Duo Zhang <zh...@apache.org>
AuthorDate: Wed Mar 13 17:21:43 2019 +0800

    HBASE-22002 Remove the deprecated methods in Admin interface
    
    Signed-off-by: Zheng Hu <op...@gmail.com>
---
 .../archetypes/exemplars/client/HelloHBase.java    |   5 +-
 .../exemplars/shaded_client/HelloHBase.java        |   5 +-
 .../hadoop/hbase/backup/impl/BackupManager.java    |  12 +-
 .../hadoop/hbase/backup/TestIncrementalBackup.java |   5 +-
 .../java/org/apache/hadoop/hbase/client/Admin.java | 961 +--------------------
 .../org/apache/hadoop/hbase/client/HBaseAdmin.java | 499 +----------
 .../org/apache/hadoop/hbase/client/HTable.java     |  66 +-
 .../hbase/security/access/AccessControlClient.java |  15 +-
 .../org/apache/hadoop/hbase/util/FutureUtils.java  |   2 +-
 .../coprocessor/TestCoprocessorTableEndpoint.java  |   2 +-
 .../hadoop/hbase/DistributedHBaseCluster.java      |   6 +-
 .../hbase/IntegrationTestDDLMasterFailover.java    |  13 +-
 .../hadoop/hbase/IntegrationTestIngestWithMOB.java |   2 +-
 .../hbase/IntegrationTestRegionReplicaPerf.java    |   2 +-
 .../apache/hadoop/hbase/chaos/actions/Action.java  |   2 +-
 .../actions/CompactRandomRegionOfTableAction.java  |   9 +-
 .../actions/FlushRandomRegionOfTableAction.java    |   9 +-
 .../MergeRandomAdjacentRegionsOfTableAction.java   |   9 +-
 .../chaos/actions/MoveRegionsOfTableAction.java    |   6 +-
 .../actions/SplitRandomRegionOfTableAction.java    |  11 +-
 .../hbase/mapreduce/IntegrationTestBulkLoad.java   |   4 +-
 .../hadoop/hbase/mttr/IntegrationTestMTTR.java     |   7 +-
 .../hbase/rsgroup/IntegrationTestRSGroup.java      |   2 +-
 .../apache/hadoop/hbase/PerformanceEvaluation.java |   2 +-
 .../hbase/mapreduce/TestHFileOutputFormat2.java    |  14 +-
 .../hadoop/hbase/snapshot/TestExportSnapshot.java  |  34 +-
 .../snapshot/TestExportSnapshotNoCluster.java      |   3 +-
 .../hbase/rest/NamespacesInstanceResource.java     |  19 +-
 .../apache/hadoop/hbase/rest/SchemaResource.java   |   4 +-
 .../hadoop/hbase/rest/PerformanceEvaluation.java   |   4 +-
 .../hadoop/hbase/coprocessor/BulkLoadObserver.java |   4 +-
 .../apache/hadoop/hbase/master/CatalogJanitor.java |   6 +-
 .../hbase/master/assignment/RegionStateStore.java  |   8 +-
 .../master/balancer/RegionLocationFinder.java      |   4 +-
 .../master/normalizer/SplitNormalizationPlan.java  |   6 +-
 .../hadoop/hbase/mob/ExpiredMobFileCleaner.java    |   6 +-
 .../hadoop/hbase/regionserver/CompactSplit.java    |   4 +-
 .../DelimitedKeyPrefixRegionSplitPolicy.java       |   4 +-
 .../hadoop/hbase/snapshot/SnapshotManifest.java    |   2 +-
 .../java/org/apache/hadoop/hbase/tool/Canary.java  |  10 +-
 .../org/apache/hadoop/hbase/util/HBaseFsck.java    |  12 +-
 .../resources/hbase-webapps/master/rsgroup.jsp     |   6 +-
 .../main/resources/hbase-webapps/master/table.jsp  |   2 +-
 .../apache/hadoop/hbase/HBaseTestingUtility.java   |  29 +-
 .../hadoop/hbase/TestClientClusterStatus.java      |  24 +-
 .../org/apache/hadoop/hbase/TestNamespace.java     |  36 +-
 .../hbase/TestPartialResultsFromClientSide.java    |   2 +-
 .../org/apache/hadoop/hbase/TestRegionLoad.java    |  17 +-
 .../CloneSnapshotFromClientNormalTestBase.java     |   2 +-
 .../client/CloneSnapshotFromClientTestBase.java    |  16 +-
 .../RestoreSnapshotFromClientSimpleTestBase.java   |   3 +-
 .../client/RestoreSnapshotFromClientTestBase.java  |  16 +-
 .../org/apache/hadoop/hbase/client/TestAdmin1.java | 292 +++----
 .../org/apache/hadoop/hbase/client/TestAdmin2.java | 140 ++-
 .../hbase/client/TestAlwaysSetScannerId.java       |   5 +-
 .../hadoop/hbase/client/TestClientTimeouts.java    |   2 +-
 .../hbase/client/TestConnectionImplementation.java |  16 +-
 .../hadoop/hbase/client/TestFromClientSide.java    |  16 +-
 .../hadoop/hbase/client/TestFromClientSide3.java   |  30 +-
 .../hbase/client/TestHBaseAdminNoCluster.java      |  20 +-
 .../hadoop/hbase/client/TestLeaseRenewal.java      |   9 +-
 .../hadoop/hbase/client/TestMetaWithReplicas.java  |   2 +-
 ...oneSnapshotFromClientCloneLinksAfterDelete.java |   5 +-
 .../hbase/client/TestMultiRespectsLimits.java      |   2 +-
 .../hbase/client/TestReplicaWithCluster.java       |   8 +-
 .../hbase/client/TestScanWithoutFetchingData.java  |   5 +-
 .../client/TestSnapshotCloneIndependence.java      |  20 +-
 .../hbase/client/TestSnapshotFromClient.java       |  43 +-
 .../hadoop/hbase/client/TestSnapshotMetadata.java  |  21 +-
 .../client/TestSnapshotTemporaryDirectory.java     |  51 +-
 .../hbase/client/TestSplitOrMergeStatus.java       |  64 +-
 .../hadoop/hbase/client/TestTableFavoredNodes.java |  28 +-
 .../hbase/client/TestTableSnapshotScanner.java     |   3 +-
 .../TestReplicationAdminWithClusters.java          |  26 +-
 .../hbase/coprocessor/TestCoprocessorMetrics.java  |   2 +-
 .../hbase/coprocessor/TestMasterObserver.java      |  10 +-
 .../io/encoding/TestLoadAndSwitchEncodeOnDisk.java |   4 +-
 .../master/TestCatalogJanitorInMemoryStates.java   |  21 +-
 .../hbase/master/TestMasterBalanceThrottling.java  |   4 +-
 .../TestMasterOperationsForRegionReplicas.java     |   6 +-
 .../hbase/master/TestMasterStatusServlet.java      |  10 +-
 .../master/assignment/TestAssignmentOnRSCrash.java |   2 +-
 .../master/assignment/TestRogueRSAssignment.java   |  16 +-
 .../balancer/TestFavoredNodeTableImport.java       |   9 +-
 .../TestFavoredStochasticBalancerPickers.java      |   2 +-
 .../TestFavoredStochasticLoadBalancer.java         |  50 +-
 .../hbase/master/locking/TestLockManager.java      |   8 +-
 .../TestSimpleRegionNormalizerOnCluster.java       |   8 +-
 .../procedure/TestCloneSnapshotProcedure.java      |   2 +-
 .../TestDeleteColumnFamilyProcedureFromClient.java |   6 +-
 .../TestMasterFailoverWithProcedures.java          |   2 +-
 .../master/procedure/TestModifyTableProcedure.java |  34 +-
 .../hbase/master/procedure/TestProcedureAdmin.java |   4 +-
 .../procedure/TestRestoreSnapshotProcedure.java    |   9 +-
 .../procedure/TestTableDDLProcedureBase.java       |   8 +-
 .../TestTableDescriptorModificationFromClient.java |  10 +-
 .../hbase/mob/compactions/TestMobCompactor.java    |  11 +-
 .../hbase/namespace/TestNamespaceAuditor.java      |  10 +-
 .../hbase/procedure/TestProcedureManager.java      |   2 +-
 .../hadoop/hbase/quotas/TestTablesWithQuotas.java  |   2 +-
 .../regionserver/TestCompactionFileNotFound.java   |   9 +-
 .../regionserver/TestEndToEndSplitTransaction.java |   3 +-
 .../TestRegionMergeTransactionOnCluster.java       |   6 +-
 .../hadoop/hbase/regionserver/TestRegionOpen.java  |   4 +-
 .../regionserver/TestRegionServerMetrics.java      |  14 -
 .../TestRegionReplicaReplicationEndpoint.java      |   6 +-
 .../security/access/TestAccessController.java      |  10 +-
 .../TestCoprocessorWhitelistMasterObserver.java    |  11 +-
 .../hbase/security/access/TestRpcAccessChecks.java |   2 +-
 .../security/visibility/TestVisibilityLabels.java  |   2 +-
 .../hbase/snapshot/SnapshotTestingUtils.java       |   2 +-
 .../snapshot/TestFlushSnapshotFromClient.java      |  42 +-
 .../TestRestoreFlushSnapshotFromClient.java        |  20 +-
 .../apache/hadoop/hbase/util/TestHBaseFsckMOB.java |   2 +-
 .../hbase/util/TestMiniClusterLoadSequential.java  |   5 +-
 .../util/hbck/OfflineMetaRebuildTestCore.java      |  12 +-
 hbase-shell/src/main/ruby/hbase/admin.rb           |  91 +-
 hbase-shell/src/main/ruby/hbase/quotas.rb          |  33 +-
 hbase-shell/src/main/ruby/hbase/security.rb        |   4 +-
 hbase-shell/src/main/ruby/hbase/taskmonitor.rb     |   2 +-
 .../hbase/thrift2/ThriftHBaseServiceHandler.java   |  10 +-
 .../hadoop/hbase/thrift2/client/ThriftAdmin.java   | 251 +-----
 122 files changed, 904 insertions(+), 2650 deletions(-)

diff --git a/hbase-archetypes/hbase-client-project/src/main/java/org/apache/hbase/archetypes/exemplars/client/HelloHBase.java b/hbase-archetypes/hbase-client-project/src/main/java/org/apache/hbase/archetypes/exemplars/client/HelloHBase.java
index 5164ab2..a9e522f 100644
--- a/hbase-archetypes/hbase-client-project/src/main/java/org/apache/hbase/archetypes/exemplars/client/HelloHBase.java
+++ b/hbase-archetypes/hbase-client-project/src/main/java/org/apache/hbase/archetypes/exemplars/client/HelloHBase.java
@@ -66,9 +66,8 @@ public final class HelloHBase {
      * CLASSPATH, to enable creation of Connection to HBase via ZooKeeper.
      */
     try (Connection connection = ConnectionFactory.createConnection();
-            Admin admin = connection.getAdmin()) {
-
-      admin.getClusterStatus(); // assure connection successfully established
+        Admin admin = connection.getAdmin()) {
+      admin.getClusterMetrics(); // assure connection successfully established
       System.out.println("\n*** Hello HBase! -- Connection has been "
               + "established via ZooKeeper!!\n");
 
diff --git a/hbase-archetypes/hbase-shaded-client-project/src/main/java/org/apache/hbase/archetypes/exemplars/shaded_client/HelloHBase.java b/hbase-archetypes/hbase-shaded-client-project/src/main/java/org/apache/hbase/archetypes/exemplars/shaded_client/HelloHBase.java
index 94a1e71..94bdf69 100644
--- a/hbase-archetypes/hbase-shaded-client-project/src/main/java/org/apache/hbase/archetypes/exemplars/shaded_client/HelloHBase.java
+++ b/hbase-archetypes/hbase-shaded-client-project/src/main/java/org/apache/hbase/archetypes/exemplars/shaded_client/HelloHBase.java
@@ -65,9 +65,8 @@ public final class HelloHBase {
      * CLASSPATH, to enable creation of Connection to HBase via ZooKeeper.
      */
     try (Connection connection = ConnectionFactory.createConnection();
-            Admin admin = connection.getAdmin()) {
-
-      admin.getClusterStatus(); // assure connection successfully established
+        Admin admin = connection.getAdmin()) {
+      admin.getClusterMetrics(); // assure connection successfully established
       System.out.println("\n*** Hello HBase! -- Connection has been "
               + "established via ZooKeeper!!\n");
 
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java
index 90677fe..d49aef2 100644
--- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java
@@ -26,11 +26,9 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.backup.BackupHFileCleaner;
 import org.apache.hadoop.hbase.backup.BackupInfo;
@@ -45,15 +43,17 @@ import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager;
 import org.apache.hadoop.hbase.backup.regionserver.LogRollRegionServerProcedureManager;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
 import org.apache.hadoop.hbase.procedure.ProcedureManagerHost;
 import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+
 /**
  * Handles backup requests, creates backup info records in backup system table to keep track of
  * backup sessions, dispatches backup request.
@@ -208,9 +208,9 @@ public class BackupManager implements Closeable {
     if (type == BackupType.FULL && (tableList == null || tableList.isEmpty())) {
       // If table list is null for full backup, which means backup all tables. Then fill the table
       // list with all user tables from meta. It no table available, throw the request exception.
-      HTableDescriptor[] htds = null;
+      List<TableDescriptor> htds = null;
       try (Admin admin = conn.getAdmin()) {
-        htds = admin.listTables();
+        htds = admin.listTableDescriptors();
       } catch (Exception e) {
         throw new BackupException(e);
       }
@@ -219,7 +219,7 @@ public class BackupManager implements Closeable {
         throw new BackupException("No table exists for full backup of all tables.");
       } else {
         tableList = new ArrayList<>();
-        for (HTableDescriptor hTableDescriptor : htds) {
+        for (TableDescriptor hTableDescriptor : htds) {
           TableName tn = hTableDescriptor.getTableName();
           if (tn.equals(BackupSystemTable.getTableName(conf))) {
             // skip backup system table
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java
index 6e15238..525845c 100644
--- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java
@@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.backup;
 
 import static org.junit.Assert.assertTrue;
 
-import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
@@ -134,8 +133,8 @@ public class TestIncrementalBackup extends TestBackupBase {
       long startSplitTime = EnvironmentEdgeManager.currentTime();
 
       try {
-        admin.splitRegion(name);
-      } catch (IOException e) {
+        admin.splitRegionAsync(name, null).get();
+      } catch (Exception e) {
         // although split fail, this may not affect following check in current API,
         // exception will be thrown.
         LOG.debug("region is not splittable, because " + e);
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
index a0c5401..b65b1c4 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
@@ -34,9 +34,6 @@ import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.CacheEvictionStats;
 import org.apache.hadoop.hbase.ClusterMetrics;
 import org.apache.hadoop.hbase.ClusterMetrics.Option;
-import org.apache.hadoop.hbase.ClusterStatus;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.NamespaceNotFoundException;
 import org.apache.hadoop.hbase.RegionMetrics;
@@ -49,7 +46,6 @@ import org.apache.hadoop.hbase.client.replication.TableCFs;
 import org.apache.hadoop.hbase.client.security.SecurityCapability;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
 import org.apache.hadoop.hbase.quotas.QuotaFilter;
-import org.apache.hadoop.hbase.quotas.QuotaRetriever;
 import org.apache.hadoop.hbase.quotas.QuotaSettings;
 import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshotView;
 import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
@@ -62,8 +58,6 @@ 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.Bytes;
-import org.apache.hadoop.hbase.util.Pair;
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
@@ -122,18 +116,6 @@ public interface Admin extends Abortable, Closeable {
   /**
    * List all the userspace tables.
    *
-   * @return an array of read-only HTableDescriptors
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated since 2.0 version and will be removed in 3.0 version.
-   *             Use {@link #listTableDescriptors()}.
-   * @see #listTableDescriptors()
-   */
-  @Deprecated
-  HTableDescriptor[] listTables() throws IOException;
-
-  /**
-   * List all the userspace tables.
-   *
    * @return a list of TableDescriptors
    * @throws IOException if a remote or network exception occurs
    */
@@ -143,85 +125,27 @@ public interface Admin extends Abortable, Closeable {
    * List all the userspace tables that match the given pattern.
    *
    * @param pattern The compiled regular expression to match against
-   * @return an array of read-only HTableDescriptors
-   * @throws IOException if a remote or network exception occurs
-   * @see #listTables()
-   * @deprecated since 2.0 version and will be removed in 3.0 version.
-   *             Use {@link #listTableDescriptors(java.util.regex.Pattern)}.
-   * @see #listTableDescriptors(Pattern)
-   */
-  @Deprecated
-  HTableDescriptor[] listTables(Pattern pattern) throws IOException;
-
-  /**
-   * List all the userspace tables that match the given pattern.
-   *
-   * @param pattern The compiled regular expression to match against
    * @return a list of TableDescriptors
    * @throws IOException if a remote or network exception occurs
-   * @see #listTables()
+   * @see #listTableDescriptors()
    */
   default List<TableDescriptor> listTableDescriptors(Pattern pattern) throws IOException {
     return listTableDescriptors(pattern, false);
   }
 
   /**
-   * List all the userspace tables matching the given regular expression.
-   *
-   * @param regex The regular expression to match against
-   * @return a list of read-only HTableDescriptors
-   * @throws IOException if a remote or network exception occurs
-   * @see #listTableDescriptors(Pattern)
-   * @deprecated since 2.0 version and will be removed in 3.0 version. Use
-   *             {@link #listTableDescriptors(Pattern)} instead.
-   */
-  @Deprecated
-  HTableDescriptor[] listTables(String regex) throws IOException;
-
-  /**
-   * List all the tables matching the given pattern.
-   *
-   * @param pattern The compiled regular expression to match against
-   * @param includeSysTables <code>false</code> to match only against userspace tables
-   * @return an array of read-only HTableDescriptors
-   * @throws IOException if a remote or network exception occurs
-   * @see #listTables()
-   * @deprecated since 2.0 version and will be removed in 3.0 version.
-   *             Use {@link #listTableDescriptors(java.util.regex.Pattern, boolean)}.
-   * @see #listTableDescriptors(java.util.regex.Pattern, boolean)
-   */
-  @Deprecated
-  HTableDescriptor[] listTables(Pattern pattern, boolean includeSysTables)
-      throws IOException;
-
-  /**
    * List all the tables matching the given pattern.
    *
    * @param pattern The compiled regular expression to match against
    * @param includeSysTables <code>false</code> to match only against userspace tables
    * @return a list of TableDescriptors
    * @throws IOException if a remote or network exception occurs
-   * @see #listTables()
+   * @see #listTableDescriptors()
    */
   List<TableDescriptor> listTableDescriptors(Pattern pattern, boolean includeSysTables)
       throws IOException;
 
   /**
-   * List all the tables matching the given pattern.
-   *
-   * @param regex The regular expression to match against
-   * @param includeSysTables <code>false</code> to match only against userspace tables
-   * @return an array of read-only HTableDescriptors
-   * @throws IOException if a remote or network exception occurs
-   * @see #listTables(java.util.regex.Pattern, boolean)
-   * @deprecated since 2.0 version and will be removed in 3.0 version.
-   *             Use {@link #listTableDescriptors(Pattern, boolean)}.
-   */
-  @Deprecated
-  HTableDescriptor[] listTables(String regex, boolean includeSysTables)
-      throws IOException;
-
-  /**
    * List all of the names of userspace tables.
    *
    * @return TableName[] table names
@@ -241,17 +165,6 @@ public interface Admin extends Abortable, Closeable {
 
   /**
    * List all of the names of userspace tables.
-   * @param regex The regular expression to match against
-   * @return TableName[] table names
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated since 2.0 version and will be removed in 3.0 version. Use
-   *             {@link #listTableNames(Pattern)} instead.
-   */
-  @Deprecated
-  TableName[] listTableNames(String regex) throws IOException;
-
-  /**
-   * List all of the names of userspace tables.
    * @param pattern The regular expression to match against
    * @param includeSysTables <code>false</code> to match only against userspace tables
    * @return TableName[] table names
@@ -261,33 +174,6 @@ public interface Admin extends Abortable, Closeable {
       throws IOException;
 
   /**
-   * List all of the names of userspace tables.
-   * @param regex The regular expression to match against
-   * @param includeSysTables <code>false</code> to match only against userspace tables
-   * @return TableName[] table names
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated since 2.0 version and will be removed in 3.0 version. Use
-   *             {@link #listTableNames(Pattern, boolean)} instead.
-   */
-  @Deprecated
-  TableName[] listTableNames(String regex, boolean includeSysTables)
-      throws IOException;
-
-  /**
-   * Get a table descriptor.
-   *
-   * @param tableName as a {@link TableName}
-   * @return the read-only tableDescriptor
-   * @throws org.apache.hadoop.hbase.TableNotFoundException
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated since 2.0 version and will be removed in 3.0 version.
-   *             Use {@link #getDescriptor(TableName)}.
-   */
-  @Deprecated
-  HTableDescriptor getTableDescriptor(TableName tableName)
-      throws TableNotFoundException, IOException;
-
-  /**
    * Get a table descriptor.
    *
    * @param tableName as a {@link TableName}
@@ -390,47 +276,7 @@ public interface Admin extends Abortable, Closeable {
   Future<Void> deleteTableAsync(TableName tableName) throws IOException;
 
   /**
-   * Deletes tables matching the passed in pattern and wait on completion. Warning: Use this method
-   * carefully, there is no prompting and the effect is immediate. Consider using {@link
-   * #listTableDescriptors(Pattern)}
-   * and {@link #deleteTable(org.apache.hadoop.hbase.TableName)}
-   *
-   * @param regex The regular expression to match table names against
-   * @return Table descriptors for tables that couldn't be deleted.
-   *         The return htds are read-only
-   * @throws IOException
-   * @see #deleteTables(java.util.regex.Pattern)
-   * @see #deleteTable(org.apache.hadoop.hbase.TableName)
-   * @deprecated since 2.0 version and will be removed in 3.0 version
-   *             This is just a trivial helper method without any magic.
-   *             Consider using {@link #listTableDescriptors(Pattern)}
-   *             and {@link #deleteTable(TableName)}
-   */
-  @Deprecated
-  HTableDescriptor[] deleteTables(String regex) throws IOException;
-
-  /**
-   * Delete tables matching the passed in pattern and wait on completion. Warning: Use this method
-   * carefully, there is no prompting and the effect is immediate. Consider using {@link
-   * #listTableDescriptors(java.util.regex.Pattern)} and
-   * {@link #deleteTable(org.apache.hadoop.hbase.TableName)}
-   *
-   * @param pattern The pattern to match table names against
-   * @return Table descriptors for tables that couldn't be deleted
-   *         The return htds are read-only
-   * @throws IOException
-   * @deprecated since 2.0 version and will be removed in 3.0 version
-   *             This is just a trivial helper method without any magic.
-   *             Consider using {@link #listTableDescriptors(java.util.regex.Pattern)}
-   *             and {@link #deleteTable(TableName)}
-   */
-  @Deprecated
-  HTableDescriptor[] deleteTables(Pattern pattern) throws IOException;
-
-  /**
-   * Truncate a table.
-   * Synchronous operation.
-   *
+   * Truncate a table. Synchronous operation.
    * @param tableName name of table to truncate
    * @param preserveSplits <code>true</code> if the splits should be preserved
    * @throws IOException if a remote or network exception occurs
@@ -484,43 +330,6 @@ public interface Admin extends Abortable, Closeable {
   Future<Void> enableTableAsync(TableName tableName) throws IOException;
 
   /**
-   * Enable tables matching the passed in pattern and wait on completion. Warning: Use this method
-   * carefully, there is no prompting and the effect is immediate. Consider using {@link
-   * #listTableDescriptors(Pattern)} and {@link #enableTable(org.apache.hadoop.hbase.TableName)}
-   *
-   * @param regex The regular expression to match table names against
-   * @throws IOException
-   * @return Table descriptors for tables that couldn't be enabled.
-   *         The return HTDs are read-only.
-   * @see #enableTables(java.util.regex.Pattern)
-   * @see #enableTable(org.apache.hadoop.hbase.TableName)
-   * @deprecated since 2.0 version and will be removed in 3.0 version
-   *             This is just a trivial helper method without any magic.
-   *             Consider using {@link #listTableDescriptors(Pattern)}
-   *             and {@link #enableTable(org.apache.hadoop.hbase.TableName)}
-   */
-  @Deprecated
-  HTableDescriptor[] enableTables(String regex) throws IOException;
-
-  /**
-   * Enable tables matching the passed in pattern and wait on completion. Warning: Use this method
-   * carefully, there is no prompting and the effect is immediate. Consider using {@link
-   * #listTableDescriptors(java.util.regex.Pattern)} and
-   * {@link #enableTable(org.apache.hadoop.hbase.TableName)}
-   *
-   * @param pattern The pattern to match table names against
-   * @throws IOException
-   * @return Table descriptors for tables that couldn't be enabled.
-   *         The return HTDs are read-only.
-   * @deprecated since 2.0 version and will be removed in 3.0 version
-   *             This is just a trivial helper method without any magic.
-   *             Consider using {@link #listTableDescriptors(java.util.regex.Pattern)}
-   *             and {@link #enableTable(org.apache.hadoop.hbase.TableName)}
-   */
-  @Deprecated
-  HTableDescriptor[] enableTables(Pattern pattern) throws IOException;
-
-  /**
    * Disable the table but does not block and wait for it to be completely disabled.
    * You can use Future.get(long, TimeUnit) to wait on the operation to complete.
    * It may throw ExecutionException if there was an error while executing the operation
@@ -548,43 +357,6 @@ public interface Admin extends Abortable, Closeable {
   }
 
   /**
-   * Disable tables matching the passed in pattern and wait on completion. Warning: Use this method
-   * carefully, there is no prompting and the effect is immediate. Consider using {@link
-   * #listTableDescriptors(Pattern)} and {@link #disableTable(org.apache.hadoop.hbase.TableName)}
-   *
-   * @param regex The regular expression to match table names against
-   * @return Table descriptors for tables that couldn't be disabled
-   *         The return htds are read-only
-   * @throws IOException
-   * @see #disableTables(java.util.regex.Pattern)
-   * @see #disableTable(org.apache.hadoop.hbase.TableName)
-   * @deprecated since 2.0 version and will be removed in 3.0 version
-   *             This is just a trivial helper method without any magic.
-   *             Consider using {@link #listTableDescriptors(Pattern)}
-   *             and {@link #disableTable(org.apache.hadoop.hbase.TableName)}
-   */
-  @Deprecated
-  HTableDescriptor[] disableTables(String regex) throws IOException;
-
-  /**
-   * Disable tables matching the passed in pattern and wait on completion. Warning: Use this method
-   * carefully, there is no prompting and the effect is immediate. Consider using {@link
-   * #listTableDescriptors(java.util.regex.Pattern)} and
-   * {@link #disableTable(org.apache.hadoop.hbase.TableName)}
-   *
-   * @param pattern The pattern to match table names against
-   * @return Table descriptors for tables that couldn't be disabled
-   *         The return htds are read-only
-   * @throws IOException
-   * @deprecated since 2.0 version and will be removed in 3.0 version
-   *             This is just a trivial helper method without any magic.
-   *             Consider using {@link #listTableDescriptors(java.util.regex.Pattern)}
-   *             and {@link #disableTable(org.apache.hadoop.hbase.TableName)}
-   */
-  @Deprecated
-  HTableDescriptor[] disableTables(Pattern pattern) throws IOException;
-
-  /**
    * @param tableName name of table to check
    * @return <code>true</code> if table is on-line
    * @throws IOException if a remote or network exception occurs
@@ -606,69 +378,9 @@ public interface Admin extends Abortable, Closeable {
   boolean isTableAvailable(TableName tableName) throws IOException;
 
   /**
-   * Use this api to check if the table has been created with the specified number of splitkeys
-   * which was used while creating the given table. Note : If this api is used after a table's
-   * region gets splitted, the api may return <code>false</code>.
-   *
-   * @param tableName name of table to check
-   * @param splitKeys keys to check if the table has been created with all split keys
-   * @throws IOException if a remote or network excpetion occurs
-   * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use {@link #isTableAvailable(TableName)}
-   */
-  @Deprecated
-  boolean isTableAvailable(TableName tableName, byte[][] splitKeys) throws IOException;
-
-  /**
-   * Get the status of an <code>alter</code> (a.k.a <code>modify</code>) command - indicates how
-   * many regions have received the updated schema Asynchronous operation.
-   *
-   * @param tableName TableName instance
-   * @return Pair indicating the number of regions updated Pair.getFirst() is the regions that are
-   * yet to be updated Pair.getSecond() is the total number of regions of the table
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated Since 2.0.0. Will be removed in 3.0.0. No longer needed now you get a Future
-   * on an operation.
-   */
-  @Deprecated
-  Pair<Integer, Integer> getAlterStatus(TableName tableName) throws IOException;
-
-  /**
-   * Get the status of <code>alter</code> (a.k.a <code>modify</code>) command - indicates how many
-   * regions have received the updated schema Asynchronous operation.
-   *
-   * @param tableName name of the table to get the status of
-   * @return Pair indicating the number of regions updated Pair.getFirst() is the regions that are
-   * yet to be updated Pair.getSecond() is the total number of regions of the table
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated Since 2.0.0. Will be removed in 3.0.0. No longer needed now you get a Future
-   * on an operation.
-   */
-  @Deprecated
-  Pair<Integer, Integer> getAlterStatus(byte[] tableName) throws IOException;
-
-  /**
-   * Add a column family to an existing table. Synchronous operation.
-   * Use {@link #addColumnFamilyAsync(TableName, ColumnFamilyDescriptor)} instead because it
-   * returns a {@link Future} from which you can learn whether success or failure.
-   *
-   * @param tableName name of the table to add column family to
-   * @param columnFamily column family descriptor of column family to be added
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated As of release 2.0.0.
-   *             This will be removed in HBase 3.0.0.
-   *             Use {@link #addColumnFamily(TableName, ColumnFamilyDescriptor)}.
-   */
-  @Deprecated
-  default void addColumn(TableName tableName, ColumnFamilyDescriptor columnFamily)
-    throws IOException {
-    addColumnFamily(tableName, columnFamily);
-  }
-
-  /**
-   * Add a column family to an existing table. Synchronous operation.
-   * Use {@link #addColumnFamilyAsync(TableName, ColumnFamilyDescriptor)} instead because it
-   * returns a {@link Future} from which you can learn whether success or failure.
-   *
+   * Add a column family to an existing table. Synchronous operation. Use
+   * {@link #addColumnFamilyAsync(TableName, ColumnFamilyDescriptor)} instead because it returns a
+   * {@link Future} from which you can learn whether success or failure.
    * @param tableName name of the table to add column family to
    * @param columnFamily column family descriptor of column family to be added
    * @throws IOException if a remote or network exception occurs
@@ -695,24 +407,9 @@ public interface Admin extends Abortable, Closeable {
       throws IOException;
 
   /**
-   * Delete a column family from a table. Synchronous operation.
-   *  Use {@link #deleteColumnFamily(TableName, byte[])} instead because it
-   * returns a {@link Future} from which you can learn whether success or failure.
-   *
-   * @param tableName name of table
-   * @param columnFamily name of column family to be deleted
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated As of release 2.0.0.
-   *             This will be removed in HBase 3.0.0.
-   *             Use {@link #deleteColumnFamily(TableName, byte[])}}.
-   */
-  @Deprecated
-  void deleteColumn(TableName tableName, byte[] columnFamily) throws IOException;
-
-  /**
-   * Delete a column family from a table. Synchronous operation.
-   * Use {@link #deleteColumnFamily(TableName, byte[])} instead because it
-   * returns a {@link Future} from which you can learn whether success or failure.
+   * Delete a column family from a table. Synchronous operation. Use
+   * {@link #deleteColumnFamily(TableName, byte[])} instead because it returns a {@link Future} from
+   * which you can learn whether success or failure.
    * @param tableName name of table
    * @param columnFamily name of column family to be deleted
    * @throws IOException if a remote or network exception occurs
@@ -745,23 +442,6 @@ public interface Admin extends Abortable, Closeable {
    * @param tableName name of table
    * @param columnFamily new column family descriptor to use
    * @throws IOException if a remote or network exception occurs
-   * @deprecated As of release 2.0.0.
-   *             This will be removed in HBase 3.0.0.
-   *             Use {@link #modifyColumnFamily(TableName, ColumnFamilyDescriptor)}.
-   */
-  @Deprecated
-  default void modifyColumn(TableName tableName, ColumnFamilyDescriptor columnFamily)
-      throws IOException {
-    modifyColumnFamily(tableName, columnFamily);
-  }
-
-  /**
-   * Modify an existing column family on a table. Synchronous operation.
-   * Use {@link #modifyColumnFamilyAsync(TableName, ColumnFamilyDescriptor)} instead because it
-   * returns a {@link Future} from which you can learn whether success or failure.
-   * @param tableName name of table
-   * @param columnFamily new column family descriptor to use
-   * @throws IOException if a remote or network exception occurs
    */
   default void modifyColumnFamily(TableName tableName, ColumnFamilyDescriptor columnFamily)
       throws IOException {
@@ -786,67 +466,6 @@ public interface Admin extends Abortable, Closeable {
       throws IOException;
 
   /**
-   * Uses {@link #unassign(byte[], boolean)} to unassign the region. For expert-admins.
-   *
-   * @param regionname region name to close
-   * @param serverName Deprecated. Not used.
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
-   *             Use {@link #unassign(byte[], boolean)}.
-   */
-  @Deprecated
-  void closeRegion(String regionname, String serverName) throws IOException;
-
-  /**
-   * Uses {@link #unassign(byte[], boolean)} to unassign the region. For expert-admins.
-   *
-   * @param regionname region name to close
-   * @param serverName Deprecated. Not used.
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
-   *             Use {@link #unassign(byte[], boolean)}.
-   */
-  @Deprecated
-  void closeRegion(byte[] regionname, String serverName) throws IOException;
-
-  /**
-   * Uses {@link #unassign(byte[], boolean)} to unassign the region. For expert-admins.
-   *
-   * @param encodedRegionName The encoded region name; i.e. the hash that makes up the region name
-   * suffix: e.g. if regionname is
-   * <code>TestTable,0094429456,1289497600452.527db22f95c8a9e0116f0cc13c680396.</code>,
-   * then the encoded region name is: <code>527db22f95c8a9e0116f0cc13c680396</code>.
-   * @param serverName Deprecated. Not used.
-   * @return Deprecated. Returns <code>true</code> always.
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
-   *             Use {@link #unassign(byte[], boolean)}.
-   */
-  @Deprecated
-  boolean closeRegionWithEncodedRegionName(String encodedRegionName, String serverName)
-      throws IOException;
-
-  /**
-   * Used {@link #unassign(byte[], boolean)} to unassign the region. For expert-admins.
-   *
-   * @param sn Deprecated. Not used.
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
-   *             (<a href="https://issues.apache.org/jira/browse/HBASE-18231">HBASE-18231</a>).
-   *             Use {@link #unassign(byte[], boolean)}.
-   */
-  @Deprecated
-  void closeRegion(final ServerName sn, final HRegionInfo hri) throws IOException;
-
-  /**
-   * Get all the online regions on a region server.
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
-   *             (<a href="https://issues.apache.org/jira/browse/HBASE-17980">HBASE-17980</a>).
-   *             Use {@link #getRegions(ServerName sn)}.
-   */
-  @Deprecated
-  List<HRegionInfo> getOnlineRegions(ServerName sn) throws IOException;
-
-  /**
    * Get all the online regions on a region server.
    *
    * @return List of {@link RegionInfo}
@@ -1020,28 +639,6 @@ public interface Admin extends Abortable, Closeable {
     throws IOException, InterruptedException;
 
   /**
-   * Compact all regions on the region server. Asynchronous operation in that this method requests
-   * that a Compaction run and then it returns. It does not wait on the completion of Compaction (it
-   * can take a while).
-   * @param sn the region server name
-   * @param major if it's major compaction
-   * @throws IOException if a remote or network exception occurs
-   * @throws InterruptedException
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0. Use
-   *             {@link #compactRegionServer(ServerName)} or
-   *             {@link #majorCompactRegionServer(ServerName)}.
-   */
-  @Deprecated
-  default void compactRegionServer(ServerName sn, boolean major) throws IOException,
-      InterruptedException {
-    if (major) {
-      majorCompactRegionServer(sn);
-    } else {
-      compactRegionServer(sn);
-    }
-  }
-
-  /**
    * Turn the compaction on or off. Disabling compactions will also interrupt any currently ongoing
    * compactions. This state is ephemeral. The setting will be lost on restart. Compaction
    * can also be enabled/disabled by modifying configuration hbase.regionserver.compaction.enabled
@@ -1119,40 +716,13 @@ public interface Admin extends Abortable, Closeable {
 
   /**
    * Turn the load balancer on or off.
-   *
-   * @param synchronous If <code>true</code>, it waits until current balance() call, if
-   * outstanding, to return.
-   * @return Previous balancer value
-   * @deprecated Since 2.0.0. Will be removed in 3.0.0.
-   * Use {@link #balancerSwitch(boolean, boolean)} instead.
-   */
-  @Deprecated
-  default boolean setBalancerRunning(boolean on, boolean synchronous) throws IOException {
-    return balancerSwitch(on, synchronous);
-  }
-
-  /**
-   * Turn the load balancer on or off.
    * @param onOrOff Set to <code>true</code> to enable, <code>false</code> to disable.
-   * @param synchronous If <code>true</code>, it waits until current balance() call, if
-   * outstanding, to return.
+   * @param synchronous If <code>true</code>, it waits until current balance() call, if outstanding,
+   *          to return.
    * @return Previous balancer value
    */
-  boolean balancerSwitch(boolean onOrOff, boolean synchronous)
-  throws IOException;
+  boolean balancerSwitch(boolean onOrOff, boolean synchronous) throws IOException;
 
-  /**
-   * Invoke the balancer.  Will run the balancer and if regions to move, it will go ahead and do the
-   * reassignments.  Can NOT run for various reasons.  Check logs.
-   *
-   * @return <code>true</code> if balancer ran, <code>false</code> otherwise.
-   * @deprecated Since 2.0.0. Will be removed in 3.0.0.
-   * Use {@link #balance()} instead.
-   */
-  @Deprecated
-  default boolean balancer() throws IOException {
-    return balance();
-  }
 
   /**
    * Invoke the balancer.  Will run the balancer and if regions to move, it will go ahead and do the
@@ -1169,21 +739,6 @@ public interface Admin extends Abortable, Closeable {
    * logs.
    * @param force whether we should force balance even if there is region in transition
    * @return <code>true</code> if balancer ran, <code>false</code> otherwise.
-   * @deprecated Since 2.0.0. Will be removed in 3.0.0.
-   * Use {@link #balance(boolean)} instead.
-   */
-  @Deprecated
-  default boolean balancer(boolean force) throws IOException {
-    return balance(force);
-  }
-
-  /**
-   * Invoke the balancer.  Will run the balancer and if regions to move, it will
-   * go ahead and do the reassignments. If there is region in transition, force parameter of true
-   * would still run balancer. Can *not* run for other reasons.  Check
-   * logs.
-   * @param force whether we should force balance even if there is region in transition
-   * @return <code>true</code> if balancer ran, <code>false</code> otherwise.
    */
   boolean balance(boolean force) throws IOException;
 
@@ -1224,33 +779,8 @@ public interface Admin extends Abortable, Closeable {
    * Turn region normalizer on or off.
    *
    * @return Previous normalizer value
-   * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use {@link #normalizerSwitch(boolean)}}
-   * instead.
-   */
-  @Deprecated
-  default boolean setNormalizerRunning(boolean on) throws IOException {
-    return normalizerSwitch(on);
-  }
-
-  /**
-   * Turn region normalizer on or off.
-   *
-   * @return Previous normalizer value
-   */
-  boolean normalizerSwitch (boolean on) throws IOException;
-
-  /**
-   * Enable/Disable the catalog janitor.
-   *
-   * @param enable if <code>true</code> enables the catalog janitor
-   * @return the previous state
-   * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use {@link #catalogJanitorSwitch(boolean)}}
-   * instead.
    */
-  @Deprecated
-  default boolean enableCatalogJanitor(boolean enable) throws IOException {
-    return catalogJanitorSwitch(enable);
-  }
+  boolean normalizerSwitch(boolean on) throws IOException;
 
   /**
    * Enable/Disable the catalog janitor/
@@ -1264,18 +794,6 @@ public interface Admin extends Abortable, Closeable {
    * Ask for a scan of the catalog table.
    *
    * @return the number of entries cleaned
-   * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use {@link #runCatalogJanitor()}}
-   * instead.
-   */
-  @Deprecated
-  default int runCatalogScan() throws IOException {
-    return runCatalogJanitor();
-  }
-
-  /**
-   * Ask for a scan of the catalog table.
-   *
-   * @return the number of entries cleaned
    */
   int runCatalogJanitor() throws IOException;
 
@@ -1288,20 +806,6 @@ public interface Admin extends Abortable, Closeable {
   /**
    * Enable/Disable the cleaner chore.
    *
-   * @param on if <code>true</code> enables the cleaner chore
-   * @return the previous state
-   * @throws IOException
-   * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use {@link #cleanerChoreSwitch(boolean)}}
-   * instead.
-   */
-  @Deprecated
-  default boolean setCleanerChoreRunning(boolean on) throws IOException {
-    return cleanerChoreSwitch(on);
-  }
-
-  /**
-   * Enable/Disable the cleaner chore.
-   *
    * @param onOrOff if <code>true</code> enables the cleaner chore
    * @return the previous state
    * @throws IOException
@@ -1323,21 +827,6 @@ public interface Admin extends Abortable, Closeable {
    */
   boolean isCleanerChoreEnabled() throws IOException;
 
-  /**
-   * Merge two regions. Asynchronous operation.
-   *
-   * @param nameOfRegionA encoded or full name of region a
-   * @param nameOfRegionB encoded or full name of region b
-   * @param forcible <code>true</code> if do a compulsory merge, otherwise we will only merge two
-   * adjacent regions
-   * @throws IOException
-   * @deprecated Since 2.0. Will be removed in 3.0. Use
-   *     {@link #mergeRegionsAsync(byte[], byte[], boolean)} instead.
-   */
-  @Deprecated
-  void mergeRegions(byte[] nameOfRegionA, byte[] nameOfRegionB,
-      boolean forcible) throws IOException;
-
 
   /**
    * Merge two regions. Asynchronous operation.
@@ -1374,17 +863,6 @@ public interface Admin extends Abortable, Closeable {
   void split(TableName tableName) throws IOException;
 
   /**
-   * Split an individual region. Asynchronous operation.
-   *
-   * @param regionName region to split
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
-   *             Use {@link #splitRegionAsync(byte[], byte[])}.
-   */
-  @Deprecated
-  void splitRegion(byte[] regionName) throws IOException;
-
-  /**
    * Split a table. Asynchronous operation.
    *
    * @param tableName table to split
@@ -1395,19 +873,6 @@ public interface Admin extends Abortable, Closeable {
 
   /**
    * Split an individual region. Asynchronous operation.
-   *
-   * @param regionName region to split
-   * @param splitPoint the explicit position to split on
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
-   *             Use {@link #splitRegionAsync(byte[], byte[])}.
-   */
-  @Deprecated
-  void splitRegion(byte[] regionName, byte[] splitPoint)
-    throws IOException;
-
-  /**
-   * Split an individual region. Asynchronous operation.
    * @param regionName region to split
    * @param splitPoint the explicit position to split on
    * @throws IOException if a remote or network exception occurs
@@ -1416,23 +881,6 @@ public interface Admin extends Abortable, Closeable {
 
   /**
    * Modify an existing table, more IRB friendly version.
-   * @param tableName name of table.
-   * @param td modified description of the table
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated since 2.0 version and will be removed in 3.0 version. use
-   *             {@link #modifyTable(TableDescriptor)}
-   */
-  @Deprecated
-  default void modifyTable(TableName tableName, TableDescriptor td) throws IOException {
-    if (!tableName.equals(td.getTableName())) {
-      throw new IllegalArgumentException("the specified table name '" + tableName +
-        "' doesn't match with the HTD one: " + td.getTableName());
-    }
-    modifyTable(td);
-  }
-
-  /**
-   * Modify an existing table, more IRB friendly version.
    * @param td modified description of the table
    * @throws IOException if a remote or network exception occurs
    */
@@ -1441,39 +889,11 @@ public interface Admin extends Abortable, Closeable {
   }
 
   /**
-   * Modify an existing table, more IRB friendly version. Asynchronous operation.  This means that
-   * it may be a while before your schema change is updated across all of the table.
-   * You can use Future.get(long, TimeUnit) to wait on the operation to complete.
-   * It may throw ExecutionException if there was an error while executing the operation
-   * or TimeoutException in case the wait timeout was not long enough to allow the
-   * operation to complete.
-   *
-   * @param tableName name of table.
-   * @param td modified description of the table
-   * @throws IOException if a remote or network exception occurs
-   * @return the result of the async modify. You can use Future.get(long, TimeUnit) to wait on the
-   *     operation to complete
-   * @deprecated since 2.0 version and will be removed in 3.0 version.
-   *             use {@link #modifyTableAsync(TableDescriptor)}
-   */
-  @Deprecated
-  default Future<Void> modifyTableAsync(TableName tableName, TableDescriptor td)
-      throws IOException {
-    if (!tableName.equals(td.getTableName())) {
-      throw new IllegalArgumentException("the specified table name '" + tableName +
-        "' doesn't match with the HTD one: " + td.getTableName());
-    }
-    return modifyTableAsync(td);
-  }
-
-  /**
-   * Modify an existing table, more IRB (ruby) friendly version. Asynchronous operation. This means that
-   * it may be a while before your schema change is updated across all of the table.
-   * You can use Future.get(long, TimeUnit) to wait on the operation to complete.
-   * It may throw ExecutionException if there was an error while executing the operation
-   * or TimeoutException in case the wait timeout was not long enough to allow the
-   * operation to complete.
-   *
+   * Modify an existing table, more IRB (ruby) friendly version. Asynchronous operation. This means
+   * that it may be a while before your schema change is updated across all of the table. You can
+   * use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw
+   * ExecutionException if there was an error while executing the operation or TimeoutException in
+   * case the wait timeout was not long enough to allow the operation to complete.
    * @param td description of the table
    * @throws IOException if a remote or network exception occurs
    * @return the result of the async modify. You can use Future.get(long, TimeUnit) to wait on the
@@ -1517,27 +937,6 @@ public interface Admin extends Abortable, Closeable {
   void stopRegionServer(String hostnamePort) throws IOException;
 
   /**
-   * Get whole cluster status, containing status about:
-   * <pre>
-   * hbase version
-   * cluster id
-   * primary/backup master(s)
-   * master's coprocessors
-   * live/dead regionservers
-   * balancer
-   * regions in transition
-   * </pre>
-   * @return cluster status
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated since 2.0 version and will be removed in 3.0 version.
-   *             use {@link #getClusterMetrics()}
-   */
-  @Deprecated
-  default ClusterStatus getClusterStatus() throws IOException {
-    return new ClusterStatus(getClusterMetrics());
-  }
-
-  /**
    * Get whole cluster metrics, containing status about:
    * <pre>
    * hbase version
@@ -1676,24 +1075,9 @@ public interface Admin extends Abortable, Closeable {
 
   /**
    * List available namespace descriptors.
-   *
    * @return List of descriptors
    */
-  NamespaceDescriptor[] listNamespaceDescriptors()
-  throws IOException;
-
-  /**
-   * Get list of table descriptors by namespace.
-   *
-   * @param name namespace name
-   * @return HTD[] the read-only tableDescriptors
-   * @throws IOException
-   * @deprecated since 2.0 version and will be removed in 3.0 version.
-   *             use {@link #listTableDescriptorsByNamespace(byte[])}
-   */
-  @Deprecated
-  HTableDescriptor[] listTableDescriptorsByNamespace(String name)
-      throws IOException;
+  NamespaceDescriptor[] listNamespaceDescriptors() throws IOException;
 
   /**
    * Get list of table descriptors by namespace.
@@ -1713,40 +1097,12 @@ public interface Admin extends Abortable, Closeable {
    * Get the regions of a given table.
    *
    * @param tableName the name of the table
-   * @return List of {@link HRegionInfo}.
-   * @throws IOException
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
-   *             (<a href="https://issues.apache.org/jira/browse/HBASE-17980">HBASE-17980</a>).
-   *             Use {@link #getRegions(TableName)}.
-   */
-  @Deprecated
-  List<HRegionInfo> getTableRegions(TableName tableName)
-    throws IOException;
-
-  /**
-   * Get the regions of a given table.
-   *
-   * @param tableName the name of the table
    * @return List of {@link RegionInfo}.
-   * @throws IOException
    */
   List<RegionInfo> getRegions(TableName tableName) throws IOException;
 
   @Override
-  void close() throws IOException;
-
-  /**
-   * Get tableDescriptors.
-   *
-   * @param tableNames List of table names
-   * @return HTD[] the read-only tableDescriptors
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated since 2.0 version and will be removed in 3.0 version.
-   *             use {@link #listTableDescriptors(List)}
-   */
-  @Deprecated
-  HTableDescriptor[] getTableDescriptorsByTableName(List<TableName> tableNames)
-    throws IOException;
+  void close();
 
   /**
    * Get tableDescriptors.
@@ -1759,19 +1115,6 @@ public interface Admin extends Abortable, Closeable {
     throws IOException;
 
   /**
-   * Get tableDescriptors.
-   *
-   * @param names List of table names
-   * @return HTD[] the read-only tableDescriptors
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated since 2.0 version and will be removed in 3.0 version.
-   *             use {@link #listTableDescriptors(List)}
-   */
-  @Deprecated
-  HTableDescriptor[] getTableDescriptors(List<String> names)
-    throws IOException;
-
-  /**
    * Abort a procedure.
    * <p/>
    * Do not use. Usually it is ignored but if not, it can do more damage than good. See hbck2.
@@ -1803,9 +1146,8 @@ public interface Admin extends Abortable, Closeable {
    * @deprecated Since 2.1.1 -- to be removed.
    */
   @Deprecated
-  Future<Boolean> abortProcedureAsync(
-    long procId,
-    boolean mayInterruptIfRunning) throws IOException;
+  Future<Boolean> abortProcedureAsync(long procId, boolean mayInterruptIfRunning)
+      throws IOException;
 
   /**
    * Get procedures.
@@ -1838,18 +1180,6 @@ public interface Admin extends Abortable, Closeable {
    * Helper that delegates to getClusterMetrics().getMasterCoprocessorNames().
    * @return an array of master coprocessors
    * @see org.apache.hadoop.hbase.ClusterMetrics#getMasterCoprocessorNames()
-   * @deprecated since 2.0 version and will be removed in 3.0 version.
-   *             use {@link #getMasterCoprocessorNames()}
-   */
-  @Deprecated
-  default String[] getMasterCoprocessors() throws IOException {
-    return getMasterCoprocessorNames().stream().toArray(size -> new String[size]);
-  }
-
-  /**
-   * Helper that delegates to getClusterMetrics().getMasterCoprocessorNames().
-   * @return an array of master coprocessors
-   * @see org.apache.hadoop.hbase.ClusterMetrics#getMasterCoprocessorNames()
    */
   default List<String> getMasterCoprocessorNames() throws IOException {
     return getClusterMetrics(EnumSet.of(Option.MASTER_COPROCESSORS))
@@ -1931,26 +1261,6 @@ public interface Admin extends Abortable, Closeable {
   }
 
   /**
-   * Create a timestamp consistent snapshot for the given table. Snapshots are considered unique
-   * based on <b>the name of the snapshot</b>. Attempts to take a snapshot with the same name (even
-   * different type or with different parameters) will fail with a {@link SnapshotCreationException}
-   * indicating the duplicate naming. Snapshot names follow the same naming constraints as tables in
-   * HBase.
-   * @param snapshotName name of the snapshot to be created
-   * @param tableName name of the table for which snapshot is created
-   * @throws IOException if a remote or network exception occurs
-   * @throws SnapshotCreationException if snapshot creation failed
-   * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
-   * @deprecated since 2.3.0, will be removed in 3.0.0. Use {@link #snapshot(String, TableName)}
-   *             instead.
-   */
-  @Deprecated
-  default void snapshot(byte[] snapshotName, TableName tableName)
-      throws IOException, SnapshotCreationException, IllegalArgumentException {
-    snapshot(Bytes.toString(snapshotName), tableName);
-  }
-
-  /**
    * Create typed snapshot of the table. Snapshots are considered unique based on <b>the name of the
    * snapshot</b>. Attempts to take a snapshot with the same name (even a different type or with
    * different parameters) will fail with a {@link SnapshotCreationException} indicating the
@@ -1978,9 +1288,8 @@ public interface Admin extends Abortable, Closeable {
    * parameters) will fail with a {@link SnapshotCreationException} indicating the duplicate naming.
    * Snapshot names follow the same naming constraints as tables in HBase. See
    * {@link org.apache.hadoop.hbase.TableName#isLegalFullyQualifiedTableName(byte[])}. You should
-   * probably use {@link #snapshot(String, org.apache.hadoop.hbase.TableName)} or
-   * {@link #snapshot(byte[], org.apache.hadoop.hbase.TableName)} unless you are sure about the type
-   * of snapshot that you want to take.
+   * probably use {@link #snapshot(String, org.apache.hadoop.hbase.TableName)} unless you are sure
+   * about the type of snapshot that you want to take.
    * @param snapshot snapshot to take
    * @throws IOException or we lose contact with the master.
    * @throws SnapshotCreationException if snapshot failed to be taken
@@ -1997,23 +1306,6 @@ public interface Admin extends Abortable, Closeable {
    * @throws IOException if the snapshot did not succeed or we lose contact with the master.
    * @throws SnapshotCreationException if snapshot creation failed
    * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
-   * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use
-   * {@link #snapshotAsync(SnapshotDescription)} instead.
-   */
-  @Deprecated
-  @SuppressWarnings("FutureReturnValueIgnored")
-  default void takeSnapshotAsync(SnapshotDescription snapshot)
-      throws IOException, SnapshotCreationException {
-    snapshotAsync(snapshot);
-  }
-
-  /**
-   * Take a snapshot without waiting for the server to complete that snapshot (asynchronous) Only a
-   * single snapshot should be taken at a time, or results may be undefined.
-   * @param snapshot snapshot to take
-   * @throws IOException if the snapshot did not succeed or we lose contact with the master.
-   * @throws SnapshotCreationException if snapshot creation failed
-   * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
    */
   Future<Void> snapshotAsync(SnapshotDescription snapshot)
       throws IOException, SnapshotCreationException;
@@ -2045,70 +1337,12 @@ public interface Admin extends Abortable, Closeable {
    * completes without problem the failsafe snapshot is deleted.
    * @param snapshotName name of the snapshot to restore
    * @throws IOException if a remote or network exception occurs
-   * @throws org.apache.hadoop.hbase.snapshot.RestoreSnapshotException if snapshot failed to be
-   *           restored
-   * @throws IllegalArgumentException if the restore request is formatted incorrectly
-   * @deprecated since 2.3.0, will be removed in 3.0.0. Use {@link #restoreSnapshot(String)}
-   *             instead.
-   */
-  @Deprecated
-  default void restoreSnapshot(byte[] snapshotName) throws IOException, RestoreSnapshotException {
-    restoreSnapshot(Bytes.toString(snapshotName));
-  }
-
-  /**
-   * Restore the specified snapshot on the original table. (The table must be disabled) If the
-   * "hbase.snapshot.restore.take.failsafe.snapshot" configuration property is set to
-   * <code>true</code>, a snapshot of the current table is taken before executing the restore
-   * operation. In case of restore failure, the failsafe snapshot will be restored. If the restore
-   * completes without problem the failsafe snapshot is deleted.
-   * @param snapshotName name of the snapshot to restore
-   * @throws IOException if a remote or network exception occurs
    * @throws RestoreSnapshotException if snapshot failed to be restored
    * @throws IllegalArgumentException if the restore request is formatted incorrectly
    */
   void restoreSnapshot(String snapshotName) throws IOException, RestoreSnapshotException;
 
   /**
-   * Restore the specified snapshot on the original table. (The table must be disabled) If the
-   * "hbase.snapshot.restore.take.failsafe.snapshot" configuration property is set to
-   * <code>true</code>, a snapshot of the current table is taken before executing the restore
-   * operation. In case of restore failure, the failsafe snapshot will be restored. If the restore
-   * completes without problem the failsafe snapshot is deleted.
-   * @param snapshotName name of the snapshot to restore
-   * @throws IOException if a remote or network exception occurs
-   * @throws RestoreSnapshotException if snapshot failed to be restored
-   * @return the result of the async restore snapshot. You can use Future.get(long, TimeUnit) to
-   *         wait on the operation to complete.
-   * @deprecated since 2.3.0, will be removed in 3.0.0. The implementation does not take care of the
-   *             failsafe property, so do not use it any more.
-   */
-  @Deprecated
-  Future<Void> restoreSnapshotAsync(String snapshotName)
-      throws IOException, RestoreSnapshotException;
-
-  /**
-   * Restore the specified snapshot on the original table. (The table must be disabled) If
-   * 'takeFailSafeSnapshot' is set to <code>true</code>, a snapshot of the current table is taken
-   * before executing the restore operation. In case of restore failure, the failsafe snapshot will
-   * be restored. If the restore completes without problem the failsafe snapshot is deleted. The
-   * failsafe snapshot name is configurable by using the property
-   * "hbase.snapshot.restore.failsafe.name".
-   * @param snapshotName name of the snapshot to restore
-   * @param takeFailSafeSnapshot <code>true</code> if the failsafe snapshot should be taken
-   * @throws IOException if a remote or network exception occurs
-   * @throws RestoreSnapshotException if snapshot failed to be restored
-   * @throws IllegalArgumentException if the restore request is formatted incorrectly
-   * @deprecated since 2.3.0, will be removed in 3.0.0. Use
-   *             {@link #restoreSnapshot(String, boolean)} instead.
-   */
-  @Deprecated
-  default void restoreSnapshot(byte[] snapshotName, boolean takeFailSafeSnapshot)
-      throws IOException, RestoreSnapshotException {
-    restoreSnapshot(Bytes.toString(snapshotName), takeFailSafeSnapshot);
-  }
-
-  /**
    * Restore the specified snapshot on the original table. (The table must be disabled) If
    * 'takeFailSafeSnapshot' is set to <code>true</code>, a snapshot of the current table is taken
    * before executing the restore operation. In case of restore failure, the failsafe snapshot will
@@ -2140,25 +1374,8 @@ public interface Admin extends Abortable, Closeable {
    * @throws RestoreSnapshotException if snapshot failed to be restored
    * @throws IllegalArgumentException if the restore request is formatted incorrectly
    */
-  void restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot,
-      boolean restoreAcl) throws IOException, RestoreSnapshotException;
-
-  /**
-   * Create a new table by cloning the snapshot content.
-   * @param snapshotName name of the snapshot to be cloned
-   * @param tableName name of the table where the snapshot will be restored
-   * @throws IOException if a remote or network exception occurs
-   * @throws TableExistsException if table to be created already exists
-   * @throws RestoreSnapshotException if snapshot failed to be cloned
-   * @throws IllegalArgumentException if the specified table has not a valid name
-   * @deprecated since 2.3.0, will be removed in 3.0.0. Use
-   *             {@link #cloneSnapshot(String, TableName)} instead.
-   */
-  @Deprecated
-  default void cloneSnapshot(byte[] snapshotName, TableName tableName)
-      throws IOException, TableExistsException, RestoreSnapshotException {
-    cloneSnapshot(Bytes.toString(snapshotName), tableName);
-  }
+  void restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot, boolean restoreAcl)
+      throws IOException, RestoreSnapshotException;
 
   /**
    * Create a new table by cloning the snapshot content.
@@ -2244,24 +1461,6 @@ public interface Admin extends Abortable, Closeable {
    * @param props Property/Value pairs of properties passing to the procedure
    * @return data returned after procedure execution. null if no return data.
    * @throws IOException
-   * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use
-   * {@link #execProcedureWithReturn(String, String, Map)} } instead.
-   */
-  @Deprecated
-  default byte[] execProcedureWithRet(String signature, String instance, Map<String, String> props)
-      throws IOException {
-    return execProcedureWithReturn(signature, instance, props);
-  }
-
-  /**
-   * Execute a distributed procedure on a cluster.
-   * @param signature A distributed procedure is uniquely identified by its signature (default the
-   *          root ZK node name of the procedure).
-   * @param instance The instance name of the procedure. For some procedures, this parameter is
-   *          optional.
-   * @param props Property/Value pairs of properties passing to the procedure
-   * @return data returned after procedure execution. null if no return data.
-   * @throws IOException
    */
   byte[] execProcedureWithReturn(String signature, String instance, Map<String, String> props)
       throws IOException;
@@ -2289,18 +1488,6 @@ public interface Admin extends Abortable, Closeable {
   List<SnapshotDescription> listSnapshots() throws IOException;
 
   /**
-   * List all the completed snapshots matching the given regular expression.
-   *
-   * @param regex The regular expression to match against
-   * @return list of SnapshotDescription
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated since 2.0 version and will be removed in 3.0 version.
-   *             Use {@link #listSnapshots(Pattern)} instead.
-   */
-  @Deprecated
-  List<SnapshotDescription> listSnapshots(String regex) throws IOException;
-
-  /**
    * List all the completed snapshots matching the given pattern.
    *
    * @param pattern The compiled regular expression to match against
@@ -2312,20 +1499,6 @@ public interface Admin extends Abortable, Closeable {
   /**
    * List all the completed snapshots matching the given table name regular expression and snapshot
    * name regular expression.
-   * @param tableNameRegex The table name regular expression to match against
-   * @param snapshotNameRegex The snapshot name regular expression to match against
-   * @return list of completed SnapshotDescription
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated since 2.0 version and will be removed in 3.0 version.
-   *             Use {@link #listTableSnapshots(Pattern, Pattern)} instead.
-   */
-  @Deprecated
-  List<SnapshotDescription> listTableSnapshots(String tableNameRegex,
-      String snapshotNameRegex) throws IOException;
-
-  /**
-   * List all the completed snapshots matching the given table name regular expression and snapshot
-   * name regular expression.
    * @param tableNamePattern The compiled table name regular expression to match against
    * @param snapshotNamePattern The compiled snapshot name regular expression to match against
    * @return list of completed SnapshotDescription
@@ -2353,17 +1526,6 @@ public interface Admin extends Abortable, Closeable {
   /**
    * Delete existing snapshots whose names match the pattern passed.
    *
-   * @param regex The regular expression to match against
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated since 2.0 version and will be removed in 3.0 version.
-   *             Use {@link #deleteSnapshots(Pattern)} instead.
-   */
-  @Deprecated
-  void deleteSnapshots(String regex) throws IOException;
-
-  /**
-   * Delete existing snapshots whose names match the pattern passed.
-   *
    * @param pattern pattern for names of the snapshot to match
    * @throws IOException if a remote or network exception occurs
    */
@@ -2372,18 +1534,6 @@ public interface Admin extends Abortable, Closeable {
   /**
    * Delete all existing snapshots matching the given table name regular expression and snapshot
    * name regular expression.
-   * @param tableNameRegex The table name regular expression to match against
-   * @param snapshotNameRegex The snapshot name regular expression to match against
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated since 2.0 version and will be removed in 3.0 version.
-   *             Use {@link #deleteTableSnapshots(Pattern, Pattern)} instead.
-   */
-  @Deprecated
-  void deleteTableSnapshots(String tableNameRegex, String snapshotNameRegex) throws IOException;
-
-  /**
-   * Delete all existing snapshots matching the given table name regular expression and snapshot
-   * name regular expression.
    * @param tableNamePattern The compiled table name regular expression to match against
    * @param snapshotNamePattern The compiled snapshot name regular expression to match against
    * @throws IOException if a remote or network exception occurs
@@ -2400,16 +1550,6 @@ public interface Admin extends Abortable, Closeable {
   void setQuota(QuotaSettings quota) throws IOException;
 
   /**
-   * Return a QuotaRetriever to list the quotas based on the filter.
-   * @param filter the quota settings filter
-   * @return the quota retriever
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use {@link #getQuota(QuotaFilter)}.
-   */
-  @Deprecated
-  QuotaRetriever getQuotaRetriever(QuotaFilter filter) throws IOException;
-
-  /**
    * List the quotas based on the filter.
    * @param filter the quota settings filter
    * @return the QuotaSetting list
@@ -2494,35 +1634,6 @@ public interface Admin extends Abortable, Closeable {
   List<SecurityCapability> getSecurityCapabilities() throws IOException;
 
   /**
-   * Turn the Split or Merge switches on or off.
-   * @param enabled enabled or not
-   * @param synchronous If <code>true</code>, it waits until current split() call, if outstanding,
-   *          to return.
-   * @param switchTypes switchType list {@link MasterSwitchType}
-   * @return Previous switch value array
-   * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use {@link #splitSwitch(boolean, boolean)}
-   *             or {@link #mergeSwitch(boolean, boolean)} instead.
-   */
-  @Deprecated
-  default boolean[] setSplitOrMergeEnabled(boolean enabled, boolean synchronous,
-      MasterSwitchType... switchTypes) throws IOException {
-    boolean[] preValues = new boolean[switchTypes.length];
-    for (int i = 0; i < switchTypes.length; i++) {
-      switch (switchTypes[i]) {
-        case SPLIT:
-          preValues[i] = splitSwitch(enabled, synchronous);
-          break;
-        case MERGE:
-          preValues[i] = mergeSwitch(enabled, synchronous);
-          break;
-        default:
-          throw new UnsupportedOperationException("Unsupported switch type:" + switchTypes[i]);
-      }
-    }
-    return preValues;
-  }
-
-  /**
    * Turn the split switch on or off.
    * @param enabled enabled or not
    * @param synchronous If <code>true</code>, it waits until current split() call, if outstanding,
@@ -2541,26 +1652,6 @@ public interface Admin extends Abortable, Closeable {
   boolean mergeSwitch(boolean enabled, boolean synchronous) throws IOException;
 
   /**
-   * Query the current state of the switch.
-   *
-   * @return <code>true</code> if the switch is enabled, <code>false</code> otherwise.
-   * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use
-   * {@link #isSplitEnabled()} or {@link #isMergeEnabled()} instead.
-   */
-  @Deprecated
-  default boolean isSplitOrMergeEnabled(MasterSwitchType switchType) throws IOException {
-    switch (switchType) {
-      case SPLIT:
-        return isSplitEnabled();
-      case MERGE:
-        return isMergeEnabled();
-      default:
-        break;
-    }
-    throw new UnsupportedOperationException("Unsupported switch type:" + switchType);
-  }
-
-  /**
    * Query the current state of the split switch.
    * @return <code>true</code> if the switch is enabled, <code>false</code> otherwise.
    */
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 1bfb7b3..3a0588b 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
@@ -45,7 +45,6 @@ import java.util.regex.Pattern;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.CacheEvictionStats;
 import org.apache.hadoop.hbase.CacheEvictionStatsBuilder;
 import org.apache.hadoop.hbase.ClusterMetrics;
@@ -54,9 +53,7 @@ import org.apache.hadoop.hbase.ClusterMetricsBuilder;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MasterNotRunningException;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
@@ -136,7 +133,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Coprocesso
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureResponse;
@@ -170,8 +166,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedu
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProceduresRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProceduresResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest;
@@ -462,49 +456,13 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public HTableDescriptor[] listTables() throws IOException {
-    return listTables((Pattern)null, false);
-  }
-
-  @Override
-  public HTableDescriptor[] listTables(Pattern pattern) throws IOException {
-    return listTables(pattern, false);
-  }
-
-  @Override
-  public HTableDescriptor[] listTables(String regex) throws IOException {
-    return listTables(Pattern.compile(regex), false);
-  }
-
-  @Override
-  public HTableDescriptor[] listTables(final Pattern pattern, final boolean includeSysTables)
-      throws IOException {
-    return executeCallable(new MasterCallable<HTableDescriptor[]>(getConnection(),
-        getRpcControllerFactory()) {
-      @Override
-      protected HTableDescriptor[] rpcCall() throws Exception {
-        GetTableDescriptorsRequest req =
-            RequestConverter.buildGetTableDescriptorsRequest(pattern, includeSysTables);
-        return ProtobufUtil.toTableDescriptorList(master.getTableDescriptors(getRpcController(),
-                req)).stream().map(ImmutableHTableDescriptor::new).toArray(HTableDescriptor[]::new);
-      }
-    });
-  }
-
-  @Override
-  public HTableDescriptor[] listTables(String regex, boolean includeSysTables)
-      throws IOException {
-    return listTables(Pattern.compile(regex), includeSysTables);
-  }
-
-  @Override
   public TableName[] listTableNames() throws IOException {
     return listTableNames((Pattern)null, false);
   }
 
   @Override
-  public TableName[] listTableNames(String regex) throws IOException {
-    return listTableNames(Pattern.compile(regex), false);
+  public TableName[] listTableNames(Pattern pattern) throws IOException {
+    return listTableNames(pattern, false);
   }
 
   @Override
@@ -522,18 +480,6 @@ public class HBaseAdmin implements Admin {
     });
   }
 
-  @Override
-  public TableName[] listTableNames(final String regex, final boolean includeSysTables)
-      throws IOException {
-    return listTableNames(Pattern.compile(regex), includeSysTables);
-  }
-
-  @Override
-  public HTableDescriptor getTableDescriptor(final TableName tableName) throws IOException {
-    return getHTableDescriptor(tableName, getConnection(), rpcCallerFactory, rpcControllerFactory,
-       operationTimeout, rpcTimeout);
-  }
-
   static TableDescriptor getTableDescriptor(final TableName tableName, Connection connection,
       RpcRetryingCallerFactory rpcCallerFactory, final RpcControllerFactory rpcControllerFactory,
       int operationTimeout, int rpcTimeout) throws IOException {
@@ -557,38 +503,6 @@ public class HBaseAdmin implements Admin {
     throw new TableNotFoundException(tableName.getNameAsString());
   }
 
-  /**
-   * @deprecated since 2.0 version and will be removed in 3.0 version.
-   *             use {@link #getTableDescriptor(TableName,
-   *             Connection, RpcRetryingCallerFactory,RpcControllerFactory,int,int)}
-   */
-  @Deprecated
-  static HTableDescriptor getHTableDescriptor(final TableName tableName, Connection connection,
-      RpcRetryingCallerFactory rpcCallerFactory, final RpcControllerFactory rpcControllerFactory,
-      int operationTimeout, int rpcTimeout) throws IOException {
-    if (tableName == null) {
-      return null;
-    }
-    HTableDescriptor htd =
-        executeCallable(new MasterCallable<HTableDescriptor>(connection, rpcControllerFactory) {
-          @Override
-          protected HTableDescriptor rpcCall() throws Exception {
-            GetTableDescriptorsRequest req =
-                RequestConverter.buildGetTableDescriptorsRequest(tableName);
-            GetTableDescriptorsResponse htds = master.getTableDescriptors(getRpcController(), req);
-            if (!htds.getTableSchemaList().isEmpty()) {
-              return new ImmutableHTableDescriptor(
-                  ProtobufUtil.toTableDescriptor(htds.getTableSchemaList().get(0)));
-            }
-            return null;
-          }
-        }, rpcCallerFactory, operationTimeout, rpcTimeout);
-    if (htd != null) {
-      return new ImmutableHTableDescriptor(htd);
-    }
-    throw new TableNotFoundException(tableName.getNameAsString());
-  }
-
   private long getPauseTime(int tries) {
     int triesCount = tries;
     if (triesCount >= HConstants.RETRY_BACKOFF.length) {
@@ -673,7 +587,7 @@ public class HBaseAdmin implements Admin {
     }
 
     @Override
-    protected TableDescriptor getTableDescriptor() {
+    protected TableDescriptor getDescriptor() {
       return desc;
     }
 
@@ -736,36 +650,6 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public HTableDescriptor[] deleteTables(String regex) throws IOException {
-    return deleteTables(Pattern.compile(regex));
-  }
-
-  /**
-   * Delete tables matching the passed in pattern and wait on completion.
-   *
-   * Warning: Use this method carefully, there is no prompting and the effect is
-   * immediate. Consider using {@link #listTables(java.util.regex.Pattern) } and
-   * {@link #deleteTable(TableName)}
-   *
-   * @param pattern The pattern to match table names against
-   * @return Table descriptors for tables that couldn't be deleted
-   * @throws IOException
-   */
-  @Override
-  public HTableDescriptor[] deleteTables(Pattern pattern) throws IOException {
-    List<HTableDescriptor> failed = new LinkedList<>();
-    for (HTableDescriptor table : listTables(pattern)) {
-      try {
-        deleteTable(table.getTableName());
-      } catch (IOException ex) {
-        LOG.info("Failed to delete table " + table.getTableName(), ex);
-        failed.add(table);
-      }
-    }
-    return failed.toArray(new HTableDescriptor[failed.size()]);
-  }
-
-  @Override
   public Future<Void> truncateTableAsync(final TableName tableName, final boolean preserveSplits)
       throws IOException {
     TruncateTableResponse response =
@@ -864,27 +748,6 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public HTableDescriptor[] enableTables(String regex) throws IOException {
-    return enableTables(Pattern.compile(regex));
-  }
-
-  @Override
-  public HTableDescriptor[] enableTables(Pattern pattern) throws IOException {
-    List<HTableDescriptor> failed = new LinkedList<>();
-    for (HTableDescriptor table : listTables(pattern)) {
-      if (isTableDisabled(table.getTableName())) {
-        try {
-          enableTable(table.getTableName());
-        } catch (IOException ex) {
-          LOG.info("Failed to enable table " + table.getTableName(), ex);
-          failed.add(table);
-        }
-      }
-    }
-    return failed.toArray(new HTableDescriptor[failed.size()]);
-  }
-
-  @Override
   public Future<Void> disableTableAsync(final TableName tableName) throws IOException {
     TableName.isLegalFullyQualifiedTableName(tableName.getName());
     DisableTableResponse response = executeCallable(
@@ -924,27 +787,6 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public HTableDescriptor[] disableTables(String regex) throws IOException {
-    return disableTables(Pattern.compile(regex));
-  }
-
-  @Override
-  public HTableDescriptor[] disableTables(Pattern pattern) throws IOException {
-    List<HTableDescriptor> failed = new LinkedList<>();
-    for (HTableDescriptor table : listTables(pattern)) {
-      if (isTableEnabled(table.getTableName())) {
-        try {
-          disableTable(table.getTableName());
-        } catch (IOException ex) {
-          LOG.info("Failed to disable table " + table.getTableName(), ex);
-          failed.add(table);
-        }
-      }
-    }
-    return failed.toArray(new HTableDescriptor[failed.size()]);
-  }
-
-  @Override
   public boolean isTableEnabled(final TableName tableName) throws IOException {
     checkTableExists(tableName);
     return executeCallable(new RpcRetryingCallable<Boolean>() {
@@ -971,33 +813,6 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public boolean isTableAvailable(TableName tableName, byte[][] splitKeys) throws IOException {
-    return connection.isTableAvailable(tableName, splitKeys);
-  }
-
-  @Override
-  public Pair<Integer, Integer> getAlterStatus(final TableName tableName) throws IOException {
-    return executeCallable(new MasterCallable<Pair<Integer, Integer>>(getConnection(),
-        getRpcControllerFactory()) {
-      @Override
-      protected Pair<Integer, Integer> rpcCall() throws Exception {
-        setPriority(tableName);
-        GetSchemaAlterStatusRequest req = RequestConverter
-            .buildGetSchemaAlterStatusRequest(tableName);
-        GetSchemaAlterStatusResponse ret = master.getSchemaAlterStatus(getRpcController(), req);
-        Pair<Integer, Integer> pair = new Pair<>(ret.getYetToUpdateRegions(),
-            ret.getTotalRegions());
-        return pair;
-      }
-    });
-  }
-
-  @Override
-  public Pair<Integer, Integer> getAlterStatus(final byte[] tableName) throws IOException {
-    return getAlterStatus(TableName.valueOf(tableName));
-  }
-
-  @Override
   public Future<Void> addColumnFamilyAsync(final TableName tableName,
       final ColumnFamilyDescriptor columnFamily) throws IOException {
     AddColumnResponse response =
@@ -1029,18 +844,6 @@ public class HBaseAdmin implements Admin {
     }
   }
 
-  /**
-   * {@inheritDoc}
-   * @deprecated Since 2.0. Will be removed in 3.0. Use
-   *     {@link #deleteColumnFamily(TableName, byte[])} instead.
-   */
-  @Override
-  @Deprecated
-  public void deleteColumn(final TableName tableName, final byte[] columnFamily)
-      throws IOException {
-    deleteColumnFamily(tableName, columnFamily);
-  }
-
   @Override
   public Future<Void> deleteColumnFamilyAsync(final TableName tableName, final byte[] columnFamily)
       throws IOException {
@@ -1107,45 +910,6 @@ public class HBaseAdmin implements Admin {
     }
   }
 
-  @Deprecated
-  @Override
-  public void closeRegion(final String regionName, final String unused) throws IOException {
-    unassign(Bytes.toBytes(regionName), true);
-  }
-
-  @Deprecated
-  @Override
-  public void closeRegion(final byte [] regionName, final String unused) throws IOException {
-    unassign(regionName, true);
-  }
-
-  @Deprecated
-  @Override
-  public boolean closeRegionWithEncodedRegionName(final String encodedRegionName,
-      final String unused) throws IOException {
-    unassign(Bytes.toBytes(encodedRegionName), true);
-    return true;
-  }
-
-  @Deprecated
-  @Override
-  public void closeRegion(final ServerName unused, final HRegionInfo hri) throws IOException {
-    unassign(hri.getRegionName(), true);
-  }
-
-  /**
-   * @param sn
-   * @return List of {@link HRegionInfo}.
-   * @throws IOException
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
-   *             Use {@link #getRegions(ServerName)}.
-   */
-  @Deprecated
-  @Override
-  public List<HRegionInfo> getOnlineRegions(final ServerName sn) throws IOException {
-    return getRegions(sn).stream().map(ImmutableHRegionInfo::new).collect(Collectors.toList());
-  }
-
   @Override
   public void flush(final TableName tableName) throws IOException {
     checkTableExists(tableName);
@@ -1666,24 +1430,6 @@ public class HBaseAdmin implements Admin {
    * @param forcible true if do a compulsory merge, otherwise we will only merge
    *          two adjacent regions
    * @throws IOException
-   * @deprecated Since 2.0. Will be removed in 3.0. Use
-   *     {@link #mergeRegionsAsync(byte[], byte[], boolean)} instead.
-   */
-  @Deprecated
-  @Override
-  public void mergeRegions(final byte[] nameOfRegionA,
-      final byte[] nameOfRegionB, final boolean forcible)
-      throws IOException {
-    mergeRegionsAsync(nameOfRegionA, nameOfRegionB, forcible);
-  }
-
-  /**
-   * Merge two regions. Asynchronous operation.
-   * @param nameOfRegionA encoded or full name of region a
-   * @param nameOfRegionB encoded or full name of region b
-   * @param forcible true if do a compulsory merge, otherwise we will only merge
-   *          two adjacent regions
-   * @throws IOException
    */
   @Override
   public Future<Void> mergeRegionsAsync(
@@ -1710,9 +1456,9 @@ public class HBaseAdmin implements Admin {
     assert(nameofRegionsToMerge.length >= 2);
     byte[][] encodedNameofRegionsToMerge = new byte[nameofRegionsToMerge.length][];
     for(int i = 0; i < nameofRegionsToMerge.length; i++) {
-      encodedNameofRegionsToMerge[i] = HRegionInfo.isEncodedRegionName(nameofRegionsToMerge[i]) ?
+      encodedNameofRegionsToMerge[i] = RegionInfo.isEncodedRegionName(nameofRegionsToMerge[i]) ?
           nameofRegionsToMerge[i] :
-          Bytes.toBytes(HRegionInfo.encodeRegionName(nameofRegionsToMerge[i]));
+          Bytes.toBytes(RegionInfo.encodeRegionName(nameofRegionsToMerge[i]));
     }
 
     TableName tableName = null;
@@ -1722,7 +1468,7 @@ public class HBaseAdmin implements Admin {
       pair = getRegion(nameofRegionsToMerge[i]);
 
       if (pair != null) {
-        if (pair.getFirst().getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) {
+        if (pair.getFirst().getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) {
           throw new IllegalArgumentException ("Can't invoke merge on non-default regions directly");
         }
         if (tableName == null) {
@@ -1811,18 +1557,17 @@ public class HBaseAdmin implements Admin {
   @Override
   public Future<Void> splitRegionAsync(byte[] regionName, byte[] splitPoint)
       throws IOException {
-    byte[] encodedNameofRegionToSplit = HRegionInfo.isEncodedRegionName(regionName) ?
-        regionName : Bytes.toBytes(HRegionInfo.encodeRegionName(regionName));
+    byte[] encodedNameofRegionToSplit = RegionInfo.isEncodedRegionName(regionName) ?
+        regionName : Bytes.toBytes(RegionInfo.encodeRegionName(regionName));
     Pair<RegionInfo, ServerName> pair = getRegion(regionName);
     if (pair != null) {
       if (pair.getFirst() != null &&
-          pair.getFirst().getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) {
+          pair.getFirst().getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) {
         throw new IllegalArgumentException ("Can't invoke split on non-default regions directly");
       }
     } else {
-      throw new UnknownRegionException (
-          "Can't invoke merge on unknown region "
-              + Bytes.toStringBinary(encodedNameofRegionToSplit));
+      throw new UnknownRegionException(
+        "Can't invoke split on unknown region " + Bytes.toStringBinary(encodedNameofRegionToSplit));
     }
 
     return splitRegionAsync(pair.getFirst(), splitPoint);
@@ -1877,11 +1622,6 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public void splitRegion(final byte[] regionName) throws IOException {
-    splitRegion(regionName, null);
-  }
-
-  @Override
   public void split(final TableName tableName, final byte[] splitPoint) throws IOException {
     checkTableExists(tableName);
     for (HRegionLocation loc : connection.locateRegions(tableName, false, false)) {
@@ -1904,23 +1644,6 @@ public class HBaseAdmin implements Admin {
     }
   }
 
-  @Override
-  public void splitRegion(final byte[] regionName, final byte [] splitPoint) throws IOException {
-    Pair<RegionInfo, ServerName> regionServerPair = getRegion(regionName);
-    if (regionServerPair == null) {
-      throw new IllegalArgumentException("Invalid region: " + Bytes.toStringBinary(regionName));
-    }
-    if (regionServerPair.getFirst() != null &&
-        regionServerPair.getFirst().getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) {
-      throw new IllegalArgumentException("Can't split replicas directly. "
-          + "Replicas are auto-split when their primary is split.");
-    }
-    if (regionServerPair.getSecond() == null) {
-      throw new NoServerForRegionException(Bytes.toStringBinary(regionName));
-    }
-    splitRegionAsync(regionServerPair.getFirst(), splitPoint);
-  }
-
   private static class ModifyTableFuture extends TableFuture<Void> {
     public ModifyTableFuture(final HBaseAdmin admin, final TableName tableName,
         final ModifyTableResponse response) {
@@ -1936,16 +1659,6 @@ public class HBaseAdmin implements Admin {
     public String getOperationType() {
       return "MODIFY";
     }
-
-    @Override
-    protected Void postOperationResult(final Void result, final long deadlineTs)
-        throws IOException, TimeoutException {
-      // The modify operation on the table is asynchronous on the server side irrespective
-      // of whether Procedure V2 is supported or not. So, we wait in the client till
-      // all regions get updated.
-      waitForSchemaUpdate(deadlineTs);
-      return result;
-    }
   }
 
   /**
@@ -1978,9 +1691,9 @@ public class HBaseAdmin implements Admin {
           ServerName sn = null;
           if (rl != null) {
             for (HRegionLocation h : rl.getRegionLocations()) {
-              if (h != null && encodedName.equals(h.getRegionInfo().getEncodedName())) {
+              if (h != null && encodedName.equals(h.getRegion().getEncodedName())) {
                 sn = h.getServerName();
-                info = h.getRegionInfo();
+                info = h.getRegion();
                 matched = true;
               }
             }
@@ -2005,13 +1718,12 @@ public class HBaseAdmin implements Admin {
    * name, the input is returned as is. We don't throw unknown
    * region exception.
    */
-  private byte[] getRegionName(
-      final byte[] regionNameOrEncodedRegionName) throws IOException {
+  private byte[] getRegionName(final byte[] regionNameOrEncodedRegionName) throws IOException {
     if (Bytes.equals(regionNameOrEncodedRegionName,
-        HRegionInfo.FIRST_META_REGIONINFO.getRegionName())
-          || Bytes.equals(regionNameOrEncodedRegionName,
-            HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes())) {
-      return HRegionInfo.FIRST_META_REGIONINFO.getRegionName();
+      RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName()) ||
+      Bytes.equals(regionNameOrEncodedRegionName,
+        RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedNameAsBytes())) {
+      return RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName();
     }
     byte[] tmp = regionNameOrEncodedRegionName;
     Pair<RegionInfo, ServerName> regionServerPair = getRegion(regionNameOrEncodedRegionName);
@@ -2273,25 +1985,6 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public HTableDescriptor[] listTableDescriptorsByNamespace(final String name) throws IOException {
-    return executeCallable(new MasterCallable<HTableDescriptor[]>(getConnection(),
-        getRpcControllerFactory()) {
-      @Override
-      protected HTableDescriptor[] rpcCall() throws Exception {
-        List<TableSchema> list =
-            master.listTableDescriptorsByNamespace(getRpcController(),
-                ListTableDescriptorsByNamespaceRequest.newBuilder().setNamespaceName(name)
-                .build()).getTableSchemaList();
-        HTableDescriptor[] res = new HTableDescriptor[list.size()];
-        for(int i=0; i < list.size(); i++) {
-          res[i] = new ImmutableHTableDescriptor(ProtobufUtil.toTableDescriptor(list.get(i)));
-        }
-        return res;
-      }
-    });
-  }
-
-  @Override
   public TableName[] listTableNamesByNamespace(final String name) throws IOException {
     return executeCallable(new MasterCallable<TableName[]>(getConnection(),
         getRpcControllerFactory()) {
@@ -2312,10 +2005,11 @@ public class HBaseAdmin implements Admin {
 
   /**
    * Is HBase available? Throw an exception if not.
+   * <p/>
+   * TODO: do not expose ZKConnectionException.
    * @param conf system configuration
    * @throws MasterNotRunningException if the master is not running.
-   * @throws ZooKeeperConnectionException if unable to connect to zookeeper. // TODO do not expose
-   *           ZKConnectionException.
+   * @throws ZooKeeperConnectionException if unable to connect to zookeeper.
    */
   public static void available(final Configuration conf)
       throws MasterNotRunningException, ZooKeeperConnectionException, IOException {
@@ -2327,59 +2021,12 @@ public class HBaseAdmin implements Admin {
     // Check ZK first.
     // If the connection exists, we may have a connection to ZK that does not work anymore
     try (ClusterConnection connection =
-        (ClusterConnection) ConnectionFactory.createConnection(copyOfConf)) {
+      (ClusterConnection) ConnectionFactory.createConnection(copyOfConf)) {
       // can throw MasterNotRunningException
       connection.isMasterRunning();
     }
   }
 
-  /**
-   *
-   * @param tableName
-   * @return List of {@link HRegionInfo}.
-   * @throws IOException
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
-   *             Use {@link #getRegions(TableName)}.
-   */
-  @Deprecated
-  @Override
-  public List<HRegionInfo> getTableRegions(final TableName tableName)
-    throws IOException {
-    return getRegions(tableName).stream()
-        .map(ImmutableHRegionInfo::new)
-        .collect(Collectors.toList());
-  }
-
-  @Override
-  public synchronized void close() throws IOException {
-  }
-
-  @Override
-  public HTableDescriptor[] getTableDescriptorsByTableName(final List<TableName> tableNames)
-  throws IOException {
-    return executeCallable(new MasterCallable<HTableDescriptor[]>(getConnection(),
-        getRpcControllerFactory()) {
-      @Override
-      protected HTableDescriptor[] rpcCall() throws Exception {
-        GetTableDescriptorsRequest req =
-            RequestConverter.buildGetTableDescriptorsRequest(tableNames);
-        return ProtobufUtil
-            .toTableDescriptorList(master.getTableDescriptors(getRpcController(), req)).stream()
-            .map(ImmutableHTableDescriptor::new).toArray(HTableDescriptor[]::new);
-      }
-    });
-  }
-
-  @Override
-  public HTableDescriptor[] getTableDescriptors(List<String> names)
-  throws IOException {
-    List<TableName> tableNames = new ArrayList<>(names.size());
-    for(String name : names) {
-      tableNames.add(TableName.valueOf(name));
-    }
-    return getTableDescriptorsByTableName(tableNames);
-  }
-
   private RollWALWriterResponse rollWALWriterImpl(final ServerName sn) throws IOException,
       FailedLogCloseException {
     final AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
@@ -2540,12 +2187,6 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public void restoreSnapshot(final byte[] snapshotName)
-      throws IOException, RestoreSnapshotException {
-    restoreSnapshot(Bytes.toString(snapshotName));
-  }
-
-  @Override
   public void restoreSnapshot(final String snapshotName)
       throws IOException, RestoreSnapshotException {
     boolean takeFailSafeSnapshot =
@@ -2554,12 +2195,6 @@ public class HBaseAdmin implements Admin {
     restoreSnapshot(snapshotName, takeFailSafeSnapshot);
   }
 
-  @Override
-  public void restoreSnapshot(final byte[] snapshotName, final boolean takeFailSafeSnapshot)
-      throws IOException, RestoreSnapshotException {
-    restoreSnapshot(Bytes.toString(snapshotName), takeFailSafeSnapshot);
-  }
-
   /**
    * Check whether the snapshot exists and contains disabled table
    *
@@ -2660,24 +2295,6 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public Future<Void> restoreSnapshotAsync(final String snapshotName)
-      throws IOException, RestoreSnapshotException {
-    TableName tableName = getTableNameBeforeRestoreSnapshot(snapshotName);
-
-    // The table does not exists, switch to clone.
-    if (!tableExists(tableName)) {
-      return cloneSnapshotAsync(snapshotName, tableName);
-    }
-
-    // Check if the table is disabled
-    if (!isTableDisabled(tableName)) {
-      throw new TableNotDisabledException(tableName);
-    }
-
-    return internalRestoreSnapshotAsync(snapshotName, tableName, false);
-  }
-
-  @Override
   public Future<Void> cloneSnapshotAsync(String snapshotName, TableName tableName,
       boolean restoreAcl) throws IOException, TableExistsException, RestoreSnapshotException {
     if (tableExists(tableName)) {
@@ -2847,11 +2464,6 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public List<SnapshotDescription> listSnapshots(String regex) throws IOException {
-    return listSnapshots(Pattern.compile(regex));
-  }
-
-  @Override
   public List<SnapshotDescription> listSnapshots(Pattern pattern) throws IOException {
     List<SnapshotDescription> matched = new LinkedList<>();
     List<SnapshotDescription> snapshots = listSnapshots();
@@ -2864,12 +2476,6 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public List<SnapshotDescription> listTableSnapshots(String tableNameRegex,
-      String snapshotNameRegex) throws IOException {
-    return listTableSnapshots(Pattern.compile(tableNameRegex), Pattern.compile(snapshotNameRegex));
-  }
-
-  @Override
   public List<SnapshotDescription> listTableSnapshots(Pattern tableNamePattern,
       Pattern snapshotNamePattern) throws IOException {
     TableName[] tableNames = listTableNames(tableNamePattern);
@@ -2910,11 +2516,6 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public void deleteSnapshots(final String regex) throws IOException {
-    deleteSnapshots(Pattern.compile(regex));
-  }
-
-  @Override
   public void deleteSnapshots(final Pattern pattern) throws IOException {
     List<SnapshotDescription> snapshots = listSnapshots(pattern);
     for (final SnapshotDescription snapshot : snapshots) {
@@ -2939,12 +2540,6 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public void deleteTableSnapshots(String tableNameRegex, String snapshotNameRegex)
-      throws IOException {
-    deleteTableSnapshots(Pattern.compile(tableNameRegex), Pattern.compile(snapshotNameRegex));
-  }
-
-  @Override
   public void deleteTableSnapshots(Pattern tableNamePattern, Pattern snapshotNamePattern)
       throws IOException {
     List<SnapshotDescription> snapshots = listTableSnapshots(tableNamePattern, snapshotNamePattern);
@@ -2970,11 +2565,6 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public QuotaRetriever getQuotaRetriever(final QuotaFilter filter) throws IOException {
-    return QuotaRetriever.open(conf, filter);
-  }
-
-  @Override
   public List<QuotaSettings> getQuota(QuotaFilter filter) throws IOException {
     List<QuotaSettings> quotas = new LinkedList<>();
     try (QuotaRetriever retriever = QuotaRetriever.open(conf, filter)) {
@@ -3035,21 +2625,6 @@ public class HBaseAdmin implements Admin {
     };
   }
 
-  /**
-   * Simple {@link Abortable}, throwing RuntimeException on abort.
-   */
-  private static class ThrowableAbortable implements Abortable {
-    @Override
-    public void abort(String why, Throwable e) {
-      throw new RuntimeException(why, e);
-    }
-
-    @Override
-    public boolean isAborted() {
-      return true;
-    }
-  }
-
   @Override
   public CoprocessorRpcChannel coprocessorService(final ServerName serverName) {
     return new SyncCoprocessorRpcChannel() {
@@ -3560,7 +3135,7 @@ public class HBaseAdmin implements Admin {
     /**
      * @return the table descriptor
      */
-    protected TableDescriptor getTableDescriptor() throws IOException {
+    protected TableDescriptor getDescriptor() throws IOException {
       return getAdmin().getDescriptor(getTableName());
     }
 
@@ -3642,19 +3217,9 @@ public class HBaseAdmin implements Admin {
       });
     }
 
-    protected void waitForSchemaUpdate(final long deadlineTs)
-        throws IOException, TimeoutException {
-      waitForState(deadlineTs, new TableWaitForStateCallable() {
-        @Override
-        public boolean checkState(int tries) throws IOException {
-          return getAdmin().getAlterStatus(tableName).getFirst() == 0;
-        }
-      });
-    }
-
     protected void waitForAllRegionsOnline(final long deadlineTs, final byte[][] splitKeys)
         throws IOException, TimeoutException {
-      final TableDescriptor desc = getTableDescriptor();
+      final TableDescriptor desc = getDescriptor();
       final AtomicInteger actualRegCount = new AtomicInteger(0);
       final MetaTableAccessor.Visitor visitor = new MetaTableAccessor.Visitor() {
         @Override
@@ -3668,16 +3233,18 @@ public class HBaseAdmin implements Admin {
           if (l == null) {
             return true;
           }
-          if (!l.getRegionInfo().getTable().equals(desc.getTableName())) {
+          if (!l.getRegion().getTable().equals(desc.getTableName())) {
             return false;
           }
-          if (l.getRegionInfo().isOffline() || l.getRegionInfo().isSplit()) return true;
+          if (l.getRegion().isOffline() || l.getRegion().isSplit()) {
+            return true;
+          }
           HRegionLocation[] locations = list.getRegionLocations();
           for (HRegionLocation location : locations) {
             if (location == null) continue;
             ServerName serverName = location.getServerName();
             // Make sure that regions are assigned to server
-            if (serverName != null && serverName.getHostAndPort() != null) {
+            if (serverName != null && serverName.getAddress() != null) {
               actualRegCount.incrementAndGet();
             }
           }
@@ -4132,7 +3699,7 @@ public class HBaseAdmin implements Admin {
       @Override
       protected List<ServerName> rpcCall() throws Exception {
         ClearDeadServersRequest req = RequestConverter.
-          buildClearDeadServersRequest(servers == null? Collections.EMPTY_LIST: servers);
+          buildClearDeadServersRequest(servers == null? Collections.emptyList(): servers);
         return ProtobufUtil.toServerNameList(
                 master.clearDeadServers(getRpcController(), req).getServerNameList());
       }
@@ -4146,7 +3713,7 @@ public class HBaseAdmin implements Admin {
     if (tableExists(newTableName)) {
       throw new TableExistsException(newTableName);
     }
-    TableDescriptor htd = TableDescriptorBuilder.copy(newTableName, getTableDescriptor(tableName));
+    TableDescriptor htd = TableDescriptorBuilder.copy(newTableName, getDescriptor(tableName));
     if (preserveSplits) {
       createTable(htd, getTableSplits(tableName));
     } else {
@@ -4296,4 +3863,8 @@ public class HBaseAdmin implements Admin {
       }
     });
   }
+
+  @Override
+  public void close() {
+  }
 }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
index 9b3afd9..23e7847 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
@@ -18,6 +18,8 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import static org.apache.hadoop.hbase.client.ConnectionUtils.checkHasFamilies;
+
 // DO NOT MAKE USE OF THESE IMPORTS! THEY ARE HERE FOR COPROCESSOR ENDPOINTS ONLY.
 // Internally, we use shaded protobuf. This below are part of our public API.
 //SEE ABOVE NOTE!
@@ -25,26 +27,44 @@ import com.google.protobuf.Descriptors;
 import com.google.protobuf.Message;
 import com.google.protobuf.Service;
 import com.google.protobuf.ServiceException;
-
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.CompareOperator;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionLocation;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.io.TimeRange;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.yetus.audience.InterfaceStability;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hbase.client.coprocessor.Batch;
 import org.apache.hadoop.hbase.client.coprocessor.Batch.Callback;
 import org.apache.hadoop.hbase.filter.BinaryComparator;
 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
+import org.apache.hadoop.hbase.io.TimeRange;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.ReflectionUtils;
+import org.apache.hadoop.hbase.util.Threads;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter;
@@ -54,27 +74,6 @@ 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.Pair;
-import org.apache.hadoop.hbase.util.ReflectionUtils;
-import org.apache.hadoop.hbase.util.Threads;
-
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.TreeMap;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Future;
-import java.util.concurrent.SynchronousQueue;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-
-import static org.apache.hadoop.hbase.client.ConnectionUtils.checkHasFamilies;
 
 /**
  * An implementation of {@link Table}. Used to communicate with a single HBase table.
@@ -224,17 +223,6 @@ public class HTable implements Table {
   }
 
   @Override
-  @Deprecated
-  public HTableDescriptor getTableDescriptor() throws IOException {
-    HTableDescriptor htd = HBaseAdmin.getHTableDescriptor(tableName, connection, rpcCallerFactory,
-      rpcControllerFactory, operationTimeoutMs, readRpcTimeoutMs);
-    if (htd != null) {
-      return new ImmutableHTableDescriptor(htd);
-    }
-    return null;
-  }
-
-  @Override
   public TableDescriptor getDescriptor() throws IOException {
     return HBaseAdmin.getTableDescriptor(tableName, connection, rpcCallerFactory,
       rpcControllerFactory, operationTimeoutMs, readRpcTimeoutMs);
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlClient.java
index 05876f0..1b1fe43 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlClient.java
@@ -21,23 +21,22 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.regex.Pattern;
-
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MasterNotRunningException;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.security.SecurityCapability;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
 import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos;
 import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService.BlockingInterface;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Utility client for doing access control admin operations.
@@ -254,7 +253,7 @@ public class AccessControlClient {
         CoprocessorRpcChannel service = table.coprocessorService(HConstants.EMPTY_START_ROW);
         BlockingInterface protocol =
             AccessControlProtos.AccessControlService.newBlockingStub(service);
-        HTableDescriptor[] htds = null;
+        List<TableDescriptor> htds = null;
         if (tableRegex == null || tableRegex.isEmpty()) {
           permList = AccessControlUtil.getUserPermissions(null, protocol, userName);
         } else if (tableRegex.charAt(0) == '@') { // Namespaces
@@ -268,8 +267,8 @@ public class AccessControlClient {
             }
           }
         } else { // Tables
-          htds = admin.listTables(Pattern.compile(tableRegex), true);
-          for (HTableDescriptor htd : htds) {
+          htds = admin.listTableDescriptors(Pattern.compile(tableRegex), true);
+          for (TableDescriptor htd : htds) {
             permList.addAll(AccessControlUtil.getUserPermissions(null, protocol, htd.getTableName(),
               null, null, userName));
           }
@@ -352,9 +351,9 @@ public class AccessControlClient {
         CoprocessorRpcChannel service = table.coprocessorService(HConstants.EMPTY_START_ROW);
         BlockingInterface protocol =
             AccessControlProtos.AccessControlService.newBlockingStub(service);
-        HTableDescriptor[] htds = admin.listTables(Pattern.compile(tableRegex), true);
+        List<TableDescriptor> htds = admin.listTableDescriptors(Pattern.compile(tableRegex), true);
         // Retrieve table permissions
-        for (HTableDescriptor htd : htds) {
+        for (TableDescriptor htd : htds) {
           permList.addAll(AccessControlUtil.getUserPermissions(null, protocol, htd.getTableName(),
             columnFamily, columnQualifier, userName));
         }
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FutureUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FutureUtils.java
index c43bda6..c0c7fbd 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FutureUtils.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FutureUtils.java
@@ -139,7 +139,7 @@ public final class FutureUtils {
   }
 
   /**
-   * A helper class for getting the result of a Future, and convert the error to an
+   * A helper class for getting the result of a Future with timeout, and convert the error to an
    * {@link IOException}.
    */
   public static <T> T get(Future<T> future, long timeout, TimeUnit unit) throws IOException {
diff --git a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorTableEndpoint.java b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorTableEndpoint.java
index 3bec203..b25c17d 100644
--- a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorTableEndpoint.java
+++ b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorTableEndpoint.java
@@ -151,7 +151,7 @@ public class TestCoprocessorTableEndpoint {
   private static void updateTable(HTableDescriptor desc) throws Exception {
     Admin admin = TEST_UTIL.getAdmin();
     admin.disableTable(desc.getTableName());
-    admin.modifyTable(desc.getTableName(), desc);
+    admin.modifyTable(desc);
     admin.enableTable(desc.getTableName());
   }
 
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java
index 30a3db9..5a5d6d0 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java
@@ -522,11 +522,7 @@ public class DistributedHBaseCluster extends HBaseCluster {
     // and the restore put the cluster back to Initial configuration, HAdmin instance will need
     // to refresh its connections (otherwise it will return incorrect information) or we can
     // point it to new instance.
-    try {
-      admin.close();
-    } catch (IOException ioe) {
-      LOG.warn("While closing the old connection", ioe);
-    }
+    admin.close();
     this.admin = this.connection.getAdmin();
     LOG.info("Added new HBaseAdmin");
     return true;
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestDDLMasterFailover.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestDDLMasterFailover.java
index 2fb12c3..61d0e7e 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestDDLMasterFailover.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestDDLMasterFailover.java
@@ -24,7 +24,7 @@ import java.util.List;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicBoolean;
-
+import java.util.regex.Pattern;
 import org.apache.commons.lang3.RandomStringUtils;
 import org.apache.commons.lang3.RandomUtils;
 import org.apache.hadoop.conf.Configuration;
@@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
@@ -151,8 +152,10 @@ public class IntegrationTestDDLMasterFailover extends IntegrationTestBase {
   public void cleanUpCluster() throws Exception {
     if (!keepObjectsAtTheEnd) {
       Admin admin = util.getAdmin();
-      admin.disableTables("ittable-\\d+");
-      admin.deleteTables("ittable-\\d+");
+      for (TableName tableName: admin.listTableNames(Pattern.compile("ittable-\\d+"))) {
+        admin.disableTable(tableName);
+        admin.deleteTable(tableName);
+      }
       NamespaceDescriptor [] nsds = admin.listNamespaceDescriptors();
       for(NamespaceDescriptor nsd: nsds) {
         if(nsd.getName().matches("itnamespace\\d+")) {
@@ -713,7 +716,7 @@ public class IntegrationTestDDLMasterFailover extends IntegrationTestBase {
         admin.modifyTable(td);
 
         // assertion
-        TableDescriptor freshTableDesc = admin.getTableDescriptor(tableName);
+        TableDescriptor freshTableDesc = admin.getDescriptor(tableName);
         ColumnFamilyDescriptor freshColumnDesc = freshTableDesc.getColumnFamily(columnDesc.getName());
         Assert.assertEquals("Encoding of column family: " + columnDesc + " was not altered",
             freshColumnDesc.getDataBlockEncoding().getId(), id);
@@ -781,7 +784,7 @@ public class IntegrationTestDDLMasterFailover extends IntegrationTestBase {
       Admin admin = connection.getAdmin();
       TableName tableName = selected.getTableName();
       try (Table table = connection.getTable(tableName)){
-        ArrayList<HRegionInfo> regionInfos = new ArrayList<>(admin.getTableRegions(
+        ArrayList<RegionInfo> regionInfos = new ArrayList<>(admin.getRegions(
             selected.getTableName()));
         int numRegions = regionInfos.size();
         // average number of rows to be added per action to each region
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithMOB.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithMOB.java
index dec1acd..1bd91fe 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithMOB.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithMOB.java
@@ -120,7 +120,7 @@ public class IntegrationTestIngestWithMOB extends IntegrationTestIngest {
     TableName tableName = getTablename();
     try (Connection connection = ConnectionFactory.createConnection();
          Admin admin = connection.getAdmin()) {
-      HTableDescriptor tableDesc = admin.getTableDescriptor(tableName);
+      HTableDescriptor tableDesc = new HTableDescriptor(admin.getDescriptor(tableName));
       LOG.info("Disabling table " + getTablename());
       admin.disableTable(tableName);
       ColumnFamilyDescriptor mobColumn = tableDesc.getColumnFamily(mobColumnFamily);
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestRegionReplicaPerf.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestRegionReplicaPerf.java
index d1f466e..4c46926 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestRegionReplicaPerf.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestRegionReplicaPerf.java
@@ -278,7 +278,7 @@ public class IntegrationTestRegionReplicaPerf extends IntegrationTestBase {
     // one last sanity check, then send in the clowns!
     assertEquals("Table must be created with DisabledRegionSplitPolicy. Broken test.",
         DisabledRegionSplitPolicy.class.getName(),
-        util.getAdmin().getTableDescriptor(tableName).getRegionSplitPolicyClassName());
+        util.getAdmin().getDescriptor(tableName).getRegionSplitPolicyClassName());
     startMonkey();
 
     // collect a baseline without region replicas.
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/Action.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/Action.java
index 6db6da0..e90a269 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/Action.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/Action.java
@@ -261,7 +261,7 @@ public class Action {
     Admin admin = this.context.getHBaseIntegrationTestingUtility().getAdmin();
     boolean result = false;
     try {
-      result = admin.balancer();
+      result = admin.balance();
     } catch (Exception e) {
       LOG.warn("Got exception while doing balance ", e);
     }
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/CompactRandomRegionOfTableAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/CompactRandomRegionOfTableAction.java
index f3fe6ff..5489668 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/CompactRandomRegionOfTableAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/CompactRandomRegionOfTableAction.java
@@ -19,13 +19,12 @@
 package org.apache.hadoop.hbase.chaos.actions;
 
 import java.util.List;
-
 import org.apache.commons.lang3.RandomUtils;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey;
 import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.RegionInfo;
 
 /**
  * Region that queues a compaction of a random region from the table.
@@ -55,14 +54,14 @@ public class CompactRandomRegionOfTableAction extends Action {
 
     LOG.info("Performing action: Compact random region of table "
       + tableName + ", major=" + major);
-    List<HRegionInfo> regions = admin.getTableRegions(tableName);
+    List<RegionInfo> regions = admin.getRegions(tableName);
     if (regions == null || regions.isEmpty()) {
       LOG.info("Table " + tableName + " doesn't have regions to compact");
       return;
     }
 
-    HRegionInfo region = PolicyBasedChaosMonkey.selectRandomItem(
-      regions.toArray(new HRegionInfo[regions.size()]));
+    RegionInfo region = PolicyBasedChaosMonkey.selectRandomItem(
+      regions.toArray(new RegionInfo[regions.size()]));
 
     try {
       if (major) {
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/FlushRandomRegionOfTableAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/FlushRandomRegionOfTableAction.java
index c5a1156..be15a12 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/FlushRandomRegionOfTableAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/FlushRandomRegionOfTableAction.java
@@ -19,12 +19,11 @@
 package org.apache.hadoop.hbase.chaos.actions;
 
 import java.util.List;
-
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey;
 import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.RegionInfo;
 
 /**
 * Action that tries to flush a random region of a table.
@@ -48,14 +47,14 @@ public class FlushRandomRegionOfTableAction extends Action {
     Admin admin = util.getAdmin();
 
     LOG.info("Performing action: Flush random region of table " + tableName);
-    List<HRegionInfo> regions = admin.getTableRegions(tableName);
+    List<RegionInfo> regions = admin.getRegions(tableName);
     if (regions == null || regions.isEmpty()) {
       LOG.info("Table " + tableName + " doesn't have regions to flush");
       return;
     }
 
-    HRegionInfo region = PolicyBasedChaosMonkey.selectRandomItem(
-      regions.toArray(new HRegionInfo[regions.size()]));
+    RegionInfo region = PolicyBasedChaosMonkey.selectRandomItem(
+      regions.toArray(new RegionInfo[regions.size()]));
     LOG.debug("Flushing region " + region.getRegionNameAsString());
     try {
       admin.flushRegion(region.getRegionName());
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MergeRandomAdjacentRegionsOfTableAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MergeRandomAdjacentRegionsOfTableAction.java
index 686e622..9a5b057 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MergeRandomAdjacentRegionsOfTableAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MergeRandomAdjacentRegionsOfTableAction.java
@@ -19,12 +19,11 @@
 package org.apache.hadoop.hbase.chaos.actions;
 
 import java.util.List;
-
 import org.apache.commons.lang3.RandomUtils;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.RegionInfo;
 
 /**
  * Action to merge regions of a table.
@@ -48,15 +47,15 @@ public class MergeRandomAdjacentRegionsOfTableAction extends Action {
     Admin admin = util.getAdmin();
 
     LOG.info("Performing action: Merge random adjacent regions of table " + tableName);
-    List<HRegionInfo> regions = admin.getTableRegions(tableName);
+    List<RegionInfo> regions = admin.getRegions(tableName);
     if (regions == null || regions.size() < 2) {
       LOG.info("Table " + tableName + " doesn't have enough regions to merge");
       return;
     }
 
     int i = RandomUtils.nextInt(0, regions.size() - 1);
-    HRegionInfo a = regions.get(i++);
-    HRegionInfo b = regions.get(i);
+    RegionInfo a = regions.get(i++);
+    RegionInfo b = regions.get(i);
     LOG.debug("Merging " + a.getRegionNameAsString() + " and " + b.getRegionNameAsString());
 
     // Don't try the merge if we're stopping
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MoveRegionsOfTableAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MoveRegionsOfTableAction.java
index 3496b9a..4c868a8 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MoveRegionsOfTableAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MoveRegionsOfTableAction.java
@@ -25,7 +25,6 @@ import java.util.EnumSet;
 import java.util.List;
 import org.apache.commons.lang3.RandomUtils;
 import org.apache.hadoop.hbase.ClusterMetrics.Option;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.chaos.factories.MonkeyConstants;
@@ -61,7 +60,7 @@ public class MoveRegionsOfTableAction extends Action {
     ServerName[] servers = getServers(admin);
 
     LOG.info("Performing action: Move regions of table {}", tableName);
-    List<HRegionInfo> regions = admin.getTableRegions(tableName);
+    List<RegionInfo> regions = admin.getRegions(tableName);
     if (regions == null || regions.isEmpty()) {
       LOG.info("Table {} doesn't have regions to move", tableName);
       return;
@@ -70,8 +69,7 @@ public class MoveRegionsOfTableAction extends Action {
     Collections.shuffle(regions);
 
     long start = System.currentTimeMillis();
-    for (HRegionInfo regionInfo:regions) {
-
+    for (RegionInfo regionInfo : regions) {
       // Don't try the move if we're stopping
       if (context.isStopping()) {
         return;
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/SplitRandomRegionOfTableAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/SplitRandomRegionOfTableAction.java
index de266f5..890ff92 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/SplitRandomRegionOfTableAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/SplitRandomRegionOfTableAction.java
@@ -19,12 +19,11 @@
 package org.apache.hadoop.hbase.chaos.actions;
 
 import java.util.List;
-
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey;
 import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.RegionInfo;
 
 /**
 * Action that tries to split a random region of a table.
@@ -48,7 +47,7 @@ public class SplitRandomRegionOfTableAction extends Action {
     Admin admin = util.getAdmin();
 
     LOG.info("Performing action: Split random region of table " + tableName);
-    List<HRegionInfo> regions = admin.getTableRegions(tableName);
+    List<RegionInfo> regions = admin.getRegions(tableName);
     if (regions == null || regions.isEmpty()) {
       LOG.info("Table " + tableName + " doesn't have regions to split");
       return;
@@ -58,11 +57,11 @@ public class SplitRandomRegionOfTableAction extends Action {
       return;
     }
 
-    HRegionInfo region = PolicyBasedChaosMonkey.selectRandomItem(
-        regions.toArray(new HRegionInfo[regions.size()]));
+    RegionInfo region = PolicyBasedChaosMonkey.selectRandomItem(
+        regions.toArray(new RegionInfo[regions.size()]));
     LOG.debug("Splitting region " + region.getRegionNameAsString());
     try {
-      admin.splitRegion(region.getRegionName());
+      admin.splitRegionAsync(region.getRegionName(), null).get();
     } catch (Exception ex) {
       LOG.warn("Split failed, might be caused by other chaos: " + ex.getMessage());
     }
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestBulkLoad.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestBulkLoad.java
index ba2e214..a28c9f6 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestBulkLoad.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestBulkLoad.java
@@ -670,9 +670,9 @@ public class IntegrationTestBulkLoad extends IntegrationTestBase {
       LOG.error("Failure in chain verification: " + msg);
       try (Connection connection = ConnectionFactory.createConnection(context.getConfiguration());
           Admin admin = connection.getAdmin()) {
-        LOG.error("cluster status:\n" + admin.getClusterStatus());
+        LOG.error("cluster metrics:\n" + admin.getClusterMetrics());
         LOG.error("table regions:\n"
-            + Joiner.on("\n").join(admin.getTableRegions(table)));
+            + Joiner.on("\n").join(admin.getRegions(table)));
       }
     }
   }
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java
index 4c2e379..e1ff49c 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java
@@ -30,7 +30,7 @@ import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import org.apache.commons.lang3.RandomStringUtils;
 import org.apache.commons.math3.stat.descriptive.DescriptiveStatistics;
-import org.apache.hadoop.hbase.ClusterStatus;
+import org.apache.hadoop.hbase.ClusterMetrics;
 import org.apache.hadoop.hbase.IntegrationTestingUtility;
 import org.apache.hadoop.hbase.InvalidFamilyOperationException;
 import org.apache.hadoop.hbase.NamespaceExistException;
@@ -59,7 +59,6 @@ import org.apache.hadoop.hbase.filter.KeyOnlyFilter;
 import org.apache.hadoop.hbase.ipc.FatalConnectionException;
 import org.apache.hadoop.hbase.regionserver.NoSuchColumnFamilyException;
 import org.apache.hadoop.hbase.security.AccessDeniedException;
-import org.apache.hbase.thirdparty.com.google.common.base.MoreObjects;
 import org.apache.hadoop.hbase.testclassification.IntegrationTests;
 import org.apache.hadoop.hbase.trace.TraceUtil;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -74,6 +73,8 @@ import org.junit.experimental.categories.Category;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hbase.thirdparty.com.google.common.base.MoreObjects;
+
 /**
  * Integration test that should benchmark how fast HBase can recover from failures. This test starts
  * different threads:
@@ -572,7 +573,7 @@ public class IntegrationTestMTTR {
       Admin admin = null;
       try {
         admin = util.getAdmin();
-        ClusterStatus status = admin.getClusterStatus();
+        ClusterMetrics status = admin.getClusterMetrics();
         return status != null;
       } finally {
         if (admin != null) {
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/rsgroup/IntegrationTestRSGroup.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/rsgroup/IntegrationTestRSGroup.java
index f5f2ff9..4f8b3ad 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/rsgroup/IntegrationTestRSGroup.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/rsgroup/IntegrationTestRSGroup.java
@@ -70,7 +70,7 @@ public class IntegrationTestRSGroup extends TestRSGroupsBase {
     deleteTableIfNecessary();
     deleteNamespaceIfNecessary();
     deleteGroups();
-    admin.setBalancerRunning(true, true);
+    admin.balancerSwitch(true, true);
 
     LOG.info("Restoring the cluster");
     ((IntegrationTestingUtility)TEST_UTIL).restoreCluster();
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
index 85897e9..2dae0e8 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
@@ -340,7 +340,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
         "Must specify an existing table for read commands. Run a write command first.");
     }
     HTableDescriptor desc =
-      exists ? admin.getTableDescriptor(TableName.valueOf(opts.tableName)) : null;
+      exists ? new HTableDescriptor(admin.getDescriptor(TableName.valueOf(opts.tableName))) : null;
     byte[][] splits = getSplits(opts);
 
     // recreate the table when user has requested presplit or when existing
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java
index e8bb805..f4f8236 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java
@@ -700,8 +700,8 @@ public class TestHFileOutputFormat2  {
       // Perform the actual load
       for (HFileOutputFormat2.TableInfo singleTableInfo : tableInfo) {
         Path tableDir = testDir;
-        String tableNameStr = singleTableInfo.getHTableDescriptor().getNameAsString();
-        LOG.info("Running LoadIncrementalHFiles on table" + tableNameStr);
+        String tableNameStr = singleTableInfo.getTableDescriptor().getTableName().getNameAsString();
+        LOG.info("Running BulkLoadHFiles on table" + tableNameStr);
         if (writeMultipleTables) {
           tableDir = new Path(testDir, tableNameStr);
         }
@@ -1237,7 +1237,7 @@ public class TestHFileOutputFormat2  {
       // deep inspection: get the StoreFile dir
       final Path storePath = new Path(
         FSUtils.getTableDir(FSUtils.getRootDir(conf), TABLE_NAMES[0]),
-          new Path(admin.getTableRegions(TABLE_NAMES[0]).get(0).getEncodedName(),
+          new Path(admin.getRegions(TABLE_NAMES[0]).get(0).getEncodedName(),
             Bytes.toString(FAMILIES[0])));
       assertEquals(0, fs.listStatus(storePath).length);
 
@@ -1318,7 +1318,7 @@ public class TestHFileOutputFormat2  {
       // deep inspection: get the StoreFile dir
       final Path storePath = new Path(
         FSUtils.getTableDir(FSUtils.getRootDir(conf), TABLE_NAMES[0]),
-          new Path(admin.getTableRegions(TABLE_NAMES[0]).get(0).getEncodedName(),
+          new Path(admin.getRegions(TABLE_NAMES[0]).get(0).getEncodedName(),
             Bytes.toString(FAMILIES[0])));
       assertEquals(0, fs.listStatus(storePath).length);
 
@@ -1411,8 +1411,10 @@ public class TestHFileOutputFormat2  {
           Admin admin = c.getAdmin();
           RegionLocator regionLocator = c.getRegionLocator(tname)) {
         Path outDir = new Path("incremental-out");
-        runIncrementalPELoad(conf, Arrays.asList(new HFileOutputFormat2.TableInfo(admin
-                .getTableDescriptor(tname), regionLocator)), outDir, false);
+        runIncrementalPELoad(conf,
+          Arrays
+            .asList(new HFileOutputFormat2.TableInfo(admin.getDescriptor(tname), regionLocator)),
+          outDir, false);
       }
     } else {
       throw new RuntimeException(
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java
index c988854..d47d60b 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java
@@ -79,8 +79,8 @@ public class TestExportSnapshot {
   public final TestName testName = new TestName();
 
   protected TableName tableName;
-  private byte[] emptySnapshotName;
-  private byte[] snapshotName;
+  private String emptySnapshotName;
+  private String snapshotName;
   private int tableNumFiles;
   private Admin admin;
 
@@ -117,8 +117,8 @@ public class TestExportSnapshot {
     this.admin = TEST_UTIL.getAdmin();
 
     tableName = TableName.valueOf("testtb-" + testName.getMethodName());
-    snapshotName = Bytes.toBytes("snaptb0-" + testName.getMethodName());
-    emptySnapshotName = Bytes.toBytes("emptySnaptb0-" + testName.getMethodName());
+    snapshotName = "snaptb0-" + testName.getMethodName();
+    emptySnapshotName = "emptySnaptb0-" + testName.getMethodName();
 
     // create Table
     createTable();
@@ -128,7 +128,7 @@ public class TestExportSnapshot {
 
     // Add some rows
     SnapshotTestingUtils.loadData(TEST_UTIL, tableName, 50, FAMILY);
-    tableNumFiles = admin.getTableRegions(tableName).size();
+    tableNumFiles = admin.getRegions(tableName).size();
 
     // take a snapshot
     admin.snapshot(snapshotName, tableName);
@@ -186,18 +186,18 @@ public class TestExportSnapshot {
 
   @Test
   public void testExportWithTargetName() throws Exception {
-    final byte[] targetName = Bytes.toBytes("testExportWithTargetName");
+    final String targetName = "testExportWithTargetName";
     testExportFileSystemState(tableName, snapshotName, targetName, tableNumFiles);
   }
 
-  private void testExportFileSystemState(final TableName tableName, final byte[] snapshotName,
-      final byte[] targetName, int filesExpected) throws Exception {
+  private void testExportFileSystemState(final TableName tableName, final String snapshotName,
+      final String targetName, int filesExpected) throws Exception {
     testExportFileSystemState(tableName, snapshotName, targetName,
       filesExpected, getHdfsDestinationDir(), false);
   }
 
   protected void testExportFileSystemState(final TableName tableName,
-      final byte[] snapshotName, final byte[] targetName, int filesExpected,
+      final String snapshotName, final String targetName, int filesExpected,
       Path copyDir, boolean overwrite) throws Exception {
     testExportFileSystemState(TEST_UTIL.getConfiguration(), tableName, snapshotName, targetName,
       filesExpected, TEST_UTIL.getDefaultRootDirPath(), copyDir,
@@ -208,7 +208,7 @@ public class TestExportSnapshot {
    * Creates destination directory, runs ExportSnapshot() tool, and runs some verifications.
    */
   protected static void testExportFileSystemState(final Configuration conf, final TableName tableName,
-      final byte[] snapshotName, final byte[] targetName, final int filesExpected,
+      final String snapshotName, final String targetName, final int filesExpected,
       final Path sourceDir, Path copyDir, final boolean overwrite,
       final RegionPredicate bypassregionPredicate, boolean success) throws Exception {
     URI hdfsUri = FileSystem.get(conf).getUri();
@@ -217,12 +217,12 @@ public class TestExportSnapshot {
 
     List<String> opts = new ArrayList<>();
     opts.add("--snapshot");
-    opts.add(Bytes.toString(snapshotName));
+    opts.add(snapshotName);
     opts.add("--copy-to");
     opts.add(copyDir.toString());
-    if (targetName != snapshotName) {
+    if (!targetName.equals(snapshotName)) {
       opts.add("--target");
-      opts.add(Bytes.toString(targetName));
+      opts.add(targetName);
     }
     if (overwrite) opts.add("--overwrite");
 
@@ -230,7 +230,7 @@ public class TestExportSnapshot {
     int res = run(conf, new ExportSnapshot(), opts.toArray(new String[opts.size()]));
     assertEquals(success ? 0 : 1, res);
     if (!success) {
-      final Path targetDir = new Path(HConstants.SNAPSHOT_DIR_NAME, Bytes.toString(targetName));
+      final Path targetDir = new Path(HConstants.SNAPSHOT_DIR_NAME, targetName);
       assertFalse(fs.exists(new Path(copyDir, targetDir)));
       return;
     }
@@ -247,12 +247,12 @@ public class TestExportSnapshot {
 
     // compare the snapshot metadata and verify the hfiles
     final FileSystem hdfs = FileSystem.get(hdfsUri, conf);
-    final Path snapshotDir = new Path(HConstants.SNAPSHOT_DIR_NAME, Bytes.toString(snapshotName));
-    final Path targetDir = new Path(HConstants.SNAPSHOT_DIR_NAME, Bytes.toString(targetName));
+    final Path snapshotDir = new Path(HConstants.SNAPSHOT_DIR_NAME, snapshotName);
+    final Path targetDir = new Path(HConstants.SNAPSHOT_DIR_NAME, targetName);
     verifySnapshotDir(hdfs, new Path(sourceDir, snapshotDir),
         fs, new Path(copyDir, targetDir));
     Set<String> snapshotFiles = verifySnapshot(conf, fs, copyDir, tableName,
-      Bytes.toString(targetName), bypassregionPredicate);
+      targetName, bypassregionPredicate);
     assertEquals(filesExpected, snapshotFiles.size());
   }
 
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshotNoCluster.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshotNoCluster.java
index 0720b36..8da5805 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshotNoCluster.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshotNoCluster.java
@@ -28,7 +28,6 @@ import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils.SnapshotMock;
 import org.apache.hadoop.hbase.testclassification.MapReduceTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.Test;
@@ -98,7 +97,7 @@ public class TestExportSnapshotNoCluster {
     builder.commit();
     int snapshotFilesCount = r1Files.length + r2Files.length;
 
-    byte[] snapshotName = Bytes.toBytes(builder.getSnapshotDescription().getName());
+    String snapshotName = builder.getSnapshotDescription().getName();
     TableName tableName = builder.getTableDescriptor().getTableName();
     TestExportSnapshot.testExportFileSystemState(TEST_UTIL.getConfiguration(),
       tableName, snapshotName, snapshotName, snapshotFilesCount,
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/NamespacesInstanceResource.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/NamespacesInstanceResource.java
index 3ff25f9..6533de1 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/NamespacesInstanceResource.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/NamespacesInstanceResource.java
@@ -19,7 +19,7 @@
 package org.apache.hadoop.hbase.rest;
 
 import java.io.IOException;
-
+import java.util.List;
 import javax.servlet.ServletContext;
 import javax.ws.rs.Consumes;
 import javax.ws.rs.DELETE;
@@ -33,16 +33,16 @@ import javax.ws.rs.core.Context;
 import javax.ws.rs.core.HttpHeaders;
 import javax.ws.rs.core.Response;
 import javax.ws.rs.core.UriInfo;
-
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.rest.model.NamespacesInstanceModel;
 import org.apache.hadoop.hbase.rest.model.TableListModel;
 import org.apache.hadoop.hbase.rest.model.TableModel;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Implements the following REST end points:
@@ -100,9 +100,10 @@ public class NamespacesInstanceResource extends ResourceBase {
     if(queryTables){
       TableListModel tableModel = new TableListModel();
       try{
-        HTableDescriptor[] tables = servlet.getAdmin().listTableDescriptorsByNamespace(namespace);
-        for(int i = 0; i < tables.length; i++){
-          tableModel.add(new TableModel(tables[i].getTableName().getQualifierAsString()));
+        List<TableDescriptor> tables =
+          servlet.getAdmin().listTableDescriptorsByNamespace(Bytes.toBytes(namespace));
+        for (TableDescriptor table : tables) {
+          tableModel.add(new TableModel(table.getTableName().getQualifierAsString()));
         }
 
         servlet.getMetrics().incrementSucessfulGetRequests(1);
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/SchemaResource.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/SchemaResource.java
index 83a70ff..786fcb6 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/SchemaResource.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/SchemaResource.java
@@ -124,7 +124,7 @@ public class SchemaResource extends ResourceBase {
       }
       if (admin.tableExists(name)) {
         admin.disableTable(name);
-        admin.modifyTable(name, htd);
+        admin.modifyTable(htd);
         admin.enableTable(name);
         servlet.getMetrics().incrementSucessfulPutRequests(1);
       } else try {
@@ -152,7 +152,7 @@ public class SchemaResource extends ResourceBase {
         .build();
     }
     try {
-      HTableDescriptor htd = admin.getTableDescriptor(name);
+      HTableDescriptor htd = new HTableDescriptor(admin.getDescriptor(name));
       admin.disableTable(name);
       try {
         for (ColumnSchemaModel family: model.getColumns()) {
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/PerformanceEvaluation.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/PerformanceEvaluation.java
index 21d25e2..c5a125f 100644
--- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/PerformanceEvaluation.java
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/PerformanceEvaluation.java
@@ -524,7 +524,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
    * @throws IOException
    */
   private boolean checkTable(RemoteAdmin admin) throws IOException {
-    HTableDescriptor tableDescriptor = getTableDescriptor();
+    HTableDescriptor tableDescriptor = getDescriptor();
     if (this.presplitRegions > 0) {
       // presplit requested
       if (admin.isTableAvailable(tableDescriptor.getTableName().getName())) {
@@ -548,7 +548,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
     return tableExists;
   }
 
-  protected HTableDescriptor getTableDescriptor() {
+  protected HTableDescriptor getDescriptor() {
     if (TABLE_DESCRIPTOR == null) {
       TABLE_DESCRIPTOR = new HTableDescriptor(tableName);
       HColumnDescriptor family = new HColumnDescriptor(FAMILY_NAME);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BulkLoadObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BulkLoadObserver.java
index b69a727..094a7d9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BulkLoadObserver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BulkLoadObserver.java
@@ -51,7 +51,7 @@ public interface BulkLoadObserver {
       * It can't bypass the default action, e.g., ctx.bypass() won't have effect.
       * If you need to get the region or table name, get it from the
       * <code>ctx</code> as follows: <code>code>ctx.getEnvironment().getRegion()</code>. Use
-      * getRegionInfo to fetch the encodedName and use getTableDescriptor() to get the tableName.
+      * getRegionInfo to fetch the encodedName and use getDescriptor() to get the tableName.
       * @param ctx the environment to interact with the framework and master
       */
     default void prePrepareBulkLoad(ObserverContext<RegionCoprocessorEnvironment> ctx)
@@ -62,7 +62,7 @@ public interface BulkLoadObserver {
       * It can't bypass the default action, e.g., ctx.bypass() won't have effect.
       * If you need to get the region or table name, get it from the
       * <code>ctx</code> as follows: <code>code>ctx.getEnvironment().getRegion()</code>. Use
-      * getRegionInfo to fetch the encodedName and use getTableDescriptor() to get the tableName.
+      * getRegionInfo to fetch the encodedName and use getDescriptor() to get the tableName.
       * @param ctx the environment to interact with the framework and master
       */
     default void preCleanupBulkLoad(ObserverContext<RegionCoprocessorEnvironment> ctx)
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java
index 73fabf8..283ebc2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java
@@ -202,7 +202,7 @@ public class CatalogJanitor extends ScheduledChore {
     FileSystem fs = this.services.getMasterFileSystem().getFileSystem();
     Path rootdir = this.services.getMasterFileSystem().getRootDir();
     Path tabledir = FSUtils.getTableDir(rootdir, mergedRegion.getTable());
-    TableDescriptor htd = getTableDescriptor(mergedRegion.getTable());
+    TableDescriptor htd = getDescriptor(mergedRegion.getTable());
     HRegionFileSystem regionFs = null;
     try {
       regionFs = HRegionFileSystem.openRegionFromFileSystem(
@@ -410,7 +410,7 @@ public class CatalogJanitor extends ScheduledChore {
     }
 
     boolean references = false;
-    TableDescriptor parentDescriptor = getTableDescriptor(parent.getTable());
+    TableDescriptor parentDescriptor = getDescriptor(parent.getTable());
     try {
       regionFs = HRegionFileSystem.openRegionFromFileSystem(
           this.services.getConfiguration(), fs, tabledir, daughter, true);
@@ -428,7 +428,7 @@ public class CatalogJanitor extends ScheduledChore {
     return new Pair<>(Boolean.TRUE, Boolean.valueOf(references));
   }
 
-  private TableDescriptor getTableDescriptor(final TableName tableName)
+  private TableDescriptor getDescriptor(final TableName tableName)
       throws FileNotFoundException, IOException {
     return this.services.getTableDescriptors().get(tableName);
   }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java
index ce4bc38..373f6d9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java
@@ -229,7 +229,7 @@ public class RegionStateStore {
   // ============================================================================================
   public void splitRegion(RegionInfo parent, RegionInfo hriA, RegionInfo hriB,
       ServerName serverName) throws IOException {
-    TableDescriptor htd = getTableDescriptor(parent.getTable());
+    TableDescriptor htd = getDescriptor(parent.getTable());
     long parentOpenSeqNum = HConstants.NO_SEQNUM;
     if (htd.hasGlobalReplicationScope()) {
       parentOpenSeqNum = getOpenSeqNumForParentRegion(parent);
@@ -243,7 +243,7 @@ public class RegionStateStore {
   // ============================================================================================
   public void mergeRegions(RegionInfo child, RegionInfo hriA, RegionInfo hriB,
       ServerName serverName) throws IOException {
-    TableDescriptor htd = getTableDescriptor(child.getTable());
+    TableDescriptor htd = getDescriptor(child.getTable());
     long regionAOpenSeqNum = -1L;
     long regionBOpenSeqNum = -1L;
     if (htd.hasGlobalReplicationScope()) {
@@ -269,7 +269,7 @@ public class RegionStateStore {
   //  Table Descriptors helpers
   // ==========================================================================
   private boolean hasGlobalReplicationScope(TableName tableName) throws IOException {
-    return hasGlobalReplicationScope(getTableDescriptor(tableName));
+    return hasGlobalReplicationScope(getDescriptor(tableName));
   }
 
   private boolean hasGlobalReplicationScope(TableDescriptor htd) {
@@ -280,7 +280,7 @@ public class RegionStateStore {
     return htd != null ? htd.getRegionReplication() : 1;
   }
 
-  private TableDescriptor getTableDescriptor(TableName tableName) throws IOException {
+  private TableDescriptor getDescriptor(TableName tableName) throws IOException {
     return master.getTableDescriptors().get(tableName);
   }
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/RegionLocationFinder.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/RegionLocationFinder.java
index fb7731f..b68c786 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/RegionLocationFinder.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/RegionLocationFinder.java
@@ -192,7 +192,7 @@ class RegionLocationFinder {
    */
   protected HDFSBlocksDistribution internalGetTopBlockLocation(RegionInfo region) {
     try {
-      TableDescriptor tableDescriptor = getTableDescriptor(region.getTable());
+      TableDescriptor tableDescriptor = getDescriptor(region.getTable());
       if (tableDescriptor != null) {
         HDFSBlocksDistribution blocksDistribution =
             HRegion.computeHDFSBlocksDistribution(getConf(), tableDescriptor, region);
@@ -213,7 +213,7 @@ class RegionLocationFinder {
    * @return TableDescriptor
    * @throws IOException
    */
-  protected TableDescriptor getTableDescriptor(TableName tableName) throws IOException {
+  protected TableDescriptor getDescriptor(TableName tableName) throws IOException {
     TableDescriptor tableDescriptor = null;
     try {
       if (this.services != null && this.services.getTableDescriptors() != null) {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SplitNormalizationPlan.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SplitNormalizationPlan.java
index b5f8e82..3899342 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SplitNormalizationPlan.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SplitNormalizationPlan.java
@@ -18,9 +18,7 @@
  */
 package org.apache.hadoop.hbase.master.normalizer;
 
-import java.io.IOException;
 import java.util.Arrays;
-
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -78,8 +76,8 @@ public class SplitNormalizationPlan implements NormalizationPlan {
   public void execute(Admin admin) {
     LOG.info("Executing splitting normalization plan: " + this);
     try {
-      admin.splitRegion(regionInfo.getRegionName());
-    } catch (IOException ex) {
+      admin.splitRegionAsync(regionInfo.getRegionName(), null).get();
+    } catch (Exception ex) {
       LOG.error("Error during region split: ", ex);
     }
   }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/ExpiredMobFileCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/ExpiredMobFileCleaner.java
index 120f11e..c1b8489 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/ExpiredMobFileCleaner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/ExpiredMobFileCleaner.java
@@ -111,11 +111,7 @@ public class ExpiredMobFileCleaner extends Configured implements Tool {
       cleanExpiredMobFiles(tableName, family);
       return 0;
     } finally {
-      try {
-        admin.close();
-      } catch (IOException e) {
-        LOG.error("Failed to close the HBaseAdmin.", e);
-      }
+      admin.close();
       try {
         connection.close();
       } catch (IOException e) {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java
index fbf73f3..ae3cc15 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java
@@ -663,8 +663,8 @@ public class CompactSplit implements CompactionRequester, PropagatingConfigurati
     @Override
     public void run() {
       Preconditions.checkNotNull(server);
-      if (server.isStopped()
-          || (region.getTableDescriptor() != null && !region.getTableDescriptor().isCompactionEnabled())) {
+      if (server.isStopped() || (region.getTableDescriptor() != null &&
+        !region.getTableDescriptor().isCompactionEnabled())) {
         region.decrementCompactionsQueuedCount();
         return;
       }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DelimitedKeyPrefixRegionSplitPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DelimitedKeyPrefixRegionSplitPolicy.java
index c90860d..8bafaf5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DelimitedKeyPrefixRegionSplitPolicy.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DelimitedKeyPrefixRegionSplitPolicy.java
@@ -53,8 +53,8 @@ public class DelimitedKeyPrefixRegionSplitPolicy extends IncreasingToUpperBoundR
     // read the prefix length from the table descriptor
     String delimiterString = region.getTableDescriptor().getValue(DELIMITER_KEY);
     if (delimiterString == null || delimiterString.length() == 0) {
-      LOG.error(DELIMITER_KEY + " not specified for table " + region.getTableDescriptor().getTableName() +
-        ". Using default RegionSplitPolicy");
+      LOG.error(DELIMITER_KEY + " not specified for table " +
+        region.getTableDescriptor().getTableName() + ". Using default RegionSplitPolicy");
       return;
     }
     delimiter = Bytes.toBytes(delimiterString);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java
index adb04c7..d8f7af6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java
@@ -130,7 +130,7 @@ public final class SnapshotManifest {
   /**
    * Return a SnapshotManifest instance with the information already loaded in-memory.
    *    SnapshotManifest manifest = SnapshotManifest.open(...)
-   *    TableDescriptor htd = manifest.getTableDescriptor()
+   *    TableDescriptor htd = manifest.getDescriptor()
    *    for (SnapshotRegionManifest regionManifest: manifest.getRegionManifests())
    *      hri = regionManifest.getRegionInfo()
    *      for (regionManifest.getFamilyFiles())
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
index 40f4aa6..cf98d4a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
@@ -90,7 +90,6 @@ import org.apache.hadoop.hbase.zookeeper.ZKConfig;
 import org.apache.hadoop.util.GenericOptionsParser;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.ZooKeeper;
@@ -99,6 +98,7 @@ import org.apache.zookeeper.data.Stat;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 
 /**
@@ -1202,13 +1202,13 @@ public final class Canary implements Tool {
 
       if (this.useRegExp) {
         Pattern pattern = null;
-        TableDescriptor[] tds = null;
+        List<TableDescriptor> tds = null;
         Set<String> tmpTables = new TreeSet<>();
         try {
           LOG.debug(String.format("reading list of tables"));
-          tds = this.admin.listTables(pattern);
+          tds = this.admin.listTableDescriptors(pattern);
           if (tds == null) {
-            tds = new TableDescriptor[0];
+            tds = Collections.emptyList();
           }
           for (String monitorTarget : monitorTargets) {
             pattern = Pattern.compile(monitorTarget);
@@ -1293,7 +1293,7 @@ public final class Canary implements Tool {
       }
       int numberOfCoveredServers = serverSet.size();
       if (numberOfCoveredServers < numberOfServers) {
-        admin.balancer();
+        admin.balance();
       }
     }
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
index 9e5f9e8..1a00945 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
@@ -2291,7 +2291,7 @@ public class HBaseFsck extends Configured implements Closeable {
     if (hi.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) {
       return;
     }
-    int numReplicas = admin.getTableDescriptor(hi.getTableName()).getRegionReplication();
+    int numReplicas = admin.getDescriptor(hi.getTableName()).getRegionReplication();
     for (int i = 1; i < numReplicas; i++) {
       if (hi.getPrimaryHRIForDeployedReplica() == null) continue;
       RegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(
@@ -2344,7 +2344,7 @@ public class HBaseFsck extends Configured implements Closeable {
     get.addColumn(HConstants.CATALOG_FAMILY, HConstants.STARTCODE_QUALIFIER);
     // also get the locations of the replicas to close if the primary region is being closed
     if (hi.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) {
-      int numReplicas = admin.getTableDescriptor(hi.getTableName()).getRegionReplication();
+      int numReplicas = admin.getDescriptor(hi.getTableName()).getRegionReplication();
       for (int i = 0; i < numReplicas; i++) {
         get.addColumn(HConstants.CATALOG_FAMILY, MetaTableAccessor.getServerColumn(i));
         get.addColumn(HConstants.CATALOG_FAMILY, MetaTableAccessor.getStartCodeColumn(i));
@@ -2395,7 +2395,7 @@ public class HBaseFsck extends Configured implements Closeable {
 
       // also assign replicas if needed (do it only when this call operates on a primary replica)
       if (hbi.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) return;
-      int replicationCount = admin.getTableDescriptor(hri.getTable()).getRegionReplication();
+      int replicationCount = admin.getDescriptor(hri.getTable()).getRegionReplication();
       for (int i = 1; i < replicationCount; i++) {
         hri = RegionReplicaUtil.getRegionInfoForReplica(hri, i);
         HbckInfo h = regionInfoMap.get(hri.getEncodedName());
@@ -2512,7 +2512,7 @@ public class HBaseFsck extends Configured implements Closeable {
           }
         }
         LOG.info("Patching hbase:meta with .regioninfo: " + hbi.getHdfsHRI());
-        int numReplicas = admin.getTableDescriptor(hbi.getTableName()).getRegionReplication();
+        int numReplicas = admin.getDescriptor(hbi.getTableName()).getRegionReplication();
         HBaseFsckRepair.fixMetaHoleOnlineAndAddReplicas(getConf(), hbi.getHdfsHRI(),
             admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
               .getLiveServerMetrics().keySet(), numReplicas);
@@ -2540,7 +2540,7 @@ public class HBaseFsck extends Configured implements Closeable {
         }
 
         LOG.info("Patching hbase:meta with with .regioninfo: " + hbi.getHdfsHRI());
-        int numReplicas = admin.getTableDescriptor(hbi.getTableName()).getRegionReplication();
+        int numReplicas = admin.getDescriptor(hbi.getTableName()).getRegionReplication();
         HBaseFsckRepair.fixMetaHoleOnlineAndAddReplicas(getConf(), hbi.getHdfsHRI(),
             admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
               .getLiveServerMetrics().keySet(), numReplicas);
@@ -3660,7 +3660,7 @@ public class HBaseFsck extends Configured implements Closeable {
         metaRegions.put(value.getReplicaId(), value);
       }
     }
-    int metaReplication = admin.getTableDescriptor(TableName.META_TABLE_NAME)
+    int metaReplication = admin.getDescriptor(TableName.META_TABLE_NAME)
         .getRegionReplication();
     boolean noProblem = true;
     // There will be always entries in regionInfoMap corresponding to hbase:meta & its replicas
diff --git a/hbase-server/src/main/resources/hbase-webapps/master/rsgroup.jsp b/hbase-server/src/main/resources/hbase-webapps/master/rsgroup.jsp
index d105e10..0b9165b 100644
--- a/hbase-server/src/main/resources/hbase-webapps/master/rsgroup.jsp
+++ b/hbase-server/src/main/resources/hbase-webapps/master/rsgroup.jsp
@@ -400,12 +400,12 @@
     </div>
 
     <% if (rsGroupTables != null && rsGroupTables.size() > 0) {
-        HTableDescriptor[] tables = null;
+        List<TableDescriptor> tables;
         try (Admin admin = master.getConnection().getAdmin()) {
-            tables = master.isInitialized() ? admin.listTables((Pattern)null, true) : null;
+            tables = master.isInitialized() ? admin.listTableDescriptors((Pattern)null, true) : null;
         }
          Map<TableName, HTableDescriptor> tableDescriptors
-            = Stream.of(tables).collect(Collectors.toMap(TableDescriptor::getTableName, p -> p));
+            = tables.stream().collect(Collectors.toMap(TableDescriptor::getTableName, p -> new HTableDescriptor(p)));
     %>
          <table class="table table-striped">
          <tr>
diff --git a/hbase-server/src/main/resources/hbase-webapps/master/table.jsp b/hbase-server/src/main/resources/hbase-webapps/master/table.jsp
index d3eaa77..ced5b44 100644
--- a/hbase-server/src/main/resources/hbase-webapps/master/table.jsp
+++ b/hbase-server/src/main/resources/hbase-webapps/master/table.jsp
@@ -172,7 +172,7 @@ if ( fqtn != null ) {
     %> Compact request accepted. <%
     } else if (action.equals("merge")) {
         if (left != null && left.length() > 0 && right != null && right.length() > 0) {
-            admin.mergeRegions(Bytes.toBytesBinary(left), Bytes.toBytesBinary(right), false);
+            admin.mergeRegionsAsync(Bytes.toBytesBinary(left), Bytes.toBytesBinary(right), false);
         }
         %> Merge request accepted. <%
     }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
index 74d168c..516c3d8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
@@ -1821,31 +1821,14 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
   }
 
   /**
-   * Modify a table, synchronous. Waiting logic similar to that of {@code admin.rb#alter_status}.
+   * Modify a table, synchronous.
+   * @deprecated just use {@link Admin#modifyTable(TableDescriptor)} directly as it is synchronous
+   *             now.
    */
-  @SuppressWarnings("serial")
+  @Deprecated
   public static void modifyTableSync(Admin admin, TableDescriptor desc)
       throws IOException, InterruptedException {
     admin.modifyTable(desc);
-    Pair<Integer, Integer> status = new Pair<Integer, Integer>() {{
-      setFirst(0);
-      setSecond(0);
-    }};
-    int i = 0;
-    do {
-      status = admin.getAlterStatus(desc.getTableName());
-      if (status.getSecond() != 0) {
-        LOG.debug(status.getSecond() - status.getFirst() + "/" + status.getSecond()
-          + " regions updated.");
-        Thread.sleep(1 * 1000L);
-      } else {
-        LOG.debug("All regions updated.");
-        break;
-      }
-    } while (status.getFirst() != 0 && i++ < 500);
-    if (status.getFirst() != 0) {
-      throw new IOException("Failed to update all regions even after 500 seconds.");
-    }
   }
 
   /**
@@ -1854,9 +1837,9 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
   public static void setReplicas(Admin admin, TableName table, int replicaCount)
       throws IOException, InterruptedException {
     admin.disableTable(table);
-    HTableDescriptor desc = new HTableDescriptor(admin.getTableDescriptor(table));
+    HTableDescriptor desc = new HTableDescriptor(admin.getDescriptor(table));
     desc.setRegionReplication(replicaCount);
-    admin.modifyTable(desc.getTableName(), desc);
+    admin.modifyTable(desc);
     admin.enableTable(table);
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterStatus.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterStatus.java
index 53fd849..635ba8b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterStatus.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterStatus.java
@@ -22,7 +22,6 @@ import java.util.EnumSet;
 import java.util.List;
 import java.util.Optional;
 import java.util.concurrent.atomic.AtomicInteger;
-import java.util.stream.Stream;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.ClusterMetrics.Option;
 import org.apache.hadoop.hbase.Waiter.Predicate;
@@ -83,25 +82,6 @@ public class TestClientClusterStatus {
   }
 
   @Test
-  public void testDefaults() throws Exception {
-    ClusterStatus origin = ADMIN.getClusterStatus();
-    ClusterStatus defaults
-        = new ClusterStatus(ADMIN.getClusterMetrics(EnumSet.allOf(Option.class)));
-    checkPbObjectNotNull(origin);
-    checkPbObjectNotNull(defaults);
-    Assert.assertEquals(origin.getHBaseVersion(), defaults.getHBaseVersion());
-    Assert.assertEquals(origin.getClusterId(), defaults.getClusterId());
-    Assert.assertTrue(origin.getAverageLoad() == defaults.getAverageLoad());
-    Assert.assertTrue(origin.getBackupMastersSize() == defaults.getBackupMastersSize());
-    Assert.assertTrue(origin.getDeadServersSize() == defaults.getDeadServersSize());
-    Assert.assertTrue(origin.getRegionsCount() == defaults.getRegionsCount());
-    Assert.assertTrue(origin.getServersSize() == defaults.getServersSize());
-    Assert.assertTrue(origin.getMasterInfoPort() == defaults.getMasterInfoPort());
-    Assert.assertTrue(origin.equals(defaults));
-    Assert.assertTrue(origin.getServersName().size() == defaults.getServersName().size());
-  }
-
-  @Test
   public void testNone() throws Exception {
     ClusterMetrics status0 = ADMIN.getClusterMetrics(EnumSet.allOf(Option.class));
     ClusterMetrics status1 = ADMIN.getClusterMetrics(EnumSet.noneOf(Option.class));
@@ -206,8 +186,8 @@ public class TestClientClusterStatus {
   public void testObserver() throws IOException {
     int preCount = MyObserver.PRE_COUNT.get();
     int postCount = MyObserver.POST_COUNT.get();
-    Assert.assertTrue(Stream.of(ADMIN.getClusterStatus().getMasterCoprocessors())
-        .anyMatch(s -> s.equals(MyObserver.class.getSimpleName())));
+    Assert.assertTrue(ADMIN.getClusterMetrics().getMasterCoprocessorNames().stream()
+      .anyMatch(s -> s.equals(MyObserver.class.getSimpleName())));
     Assert.assertEquals(preCount + 1, MyObserver.PRE_COUNT.get());
     Assert.assertEquals(postCount + 1, MyObserver.POST_COUNT.get());
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java
index 5d5bc50..f062e58 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java
@@ -23,14 +23,17 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
+import java.util.List;
 import java.util.Set;
 import java.util.concurrent.Callable;
+import java.util.regex.Pattern;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
@@ -84,7 +87,7 @@ public class TestNamespace {
 
   @Before
   public void beforeMethod() throws IOException {
-    for (HTableDescriptor desc : admin.listTables(prefix+".*")) {
+    for (TableDescriptor desc : admin.listTableDescriptors(Pattern.compile(prefix + ".*"))) {
       admin.disableTable(desc.getTableName());
       admin.deleteTable(desc.getTableName());
     }
@@ -112,14 +115,14 @@ public class TestNamespace {
     //verify existence of system tables
     Set<TableName> systemTables = Sets.newHashSet(
         TableName.META_TABLE_NAME);
-    HTableDescriptor[] descs =
-        admin.listTableDescriptorsByNamespace(NamespaceDescriptor.SYSTEM_NAMESPACE.getName());
-    assertEquals(systemTables.size(), descs.length);
-    for (HTableDescriptor desc : descs) {
+    List<TableDescriptor> descs = admin.listTableDescriptorsByNamespace(
+      Bytes.toBytes(NamespaceDescriptor.SYSTEM_NAMESPACE.getName()));
+    assertEquals(systemTables.size(), descs.size());
+    for (TableDescriptor desc : descs) {
       assertTrue(systemTables.contains(desc.getTableName()));
     }
     //verify system tables aren't listed
-    assertEquals(0, admin.listTables().length);
+    assertEquals(0, admin.listTableDescriptors().size());
 
     //Try creating default and system namespaces.
     boolean exceptionCaught = false;
@@ -189,15 +192,15 @@ public class TestNamespace {
     admin.createNamespace(NamespaceDescriptor.create(nsName).build());
     TEST_UTIL.createTable(tableName, Bytes.toBytes(nsName));
     TEST_UTIL.createTable(tableNameFoo,Bytes.toBytes(nsName));
-    assertEquals(2, admin.listTables().length);
+    assertEquals(2, admin.listTableDescriptors().size());
     assertNotNull(admin
-        .getTableDescriptor(tableName));
+        .getDescriptor(tableName));
     assertNotNull(admin
-        .getTableDescriptor(tableNameFoo));
+        .getDescriptor(tableNameFoo));
     //remove namespace and verify
     admin.disableTable(tableName);
     admin.deleteTable(tableName);
-    assertEquals(1, admin.listTables().length);
+    assertEquals(1, admin.listTableDescriptors().size());
   }
 
   @Test
@@ -222,7 +225,7 @@ public class TestNamespace {
         new Path(master.getMasterFileSystem().getRootDir(),
             new Path(HConstants.BASE_NAMESPACE_DIR,
                 new Path(nsName, desc.getTableName().getQualifierAsString())))));
-    assertEquals(1, admin.listTables().length);
+    assertEquals(1, admin.listTableDescriptors().size());
 
     //verify non-empty namespace can't be removed
     try {
@@ -253,7 +256,7 @@ public class TestNamespace {
     HColumnDescriptor colDesc = new HColumnDescriptor("cf1");
     desc.addFamily(colDesc);
     admin.createTable(desc);
-    assertTrue(admin.listTables().length == 1);
+    assertTrue(admin.listTableDescriptors().size() == 1);
     admin.disableTable(desc.getTableName());
     admin.deleteTable(desc.getTableName());
   }
@@ -265,7 +268,7 @@ public class TestNamespace {
     HColumnDescriptor colDesc = new HColumnDescriptor("cf1");
     desc.addFamily(colDesc);
     admin.createTable(desc);
-    assertEquals(0, admin.listTables().length);
+    assertEquals(0, admin.listTableDescriptors().size());
     assertTrue(admin.tableExists(tableName));
     admin.disableTable(desc.getTableName());
     admin.deleteTable(desc.getTableName());
@@ -338,15 +341,16 @@ public class TestNamespace {
     HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(prefix + "ns1", name.getMethodName()));
     htd.addFamily(new HColumnDescriptor("family1"));
     admin.createTable(htd);
-    HTableDescriptor[] htds = admin.listTableDescriptorsByNamespace(prefix + "ns1");
+    List<TableDescriptor> htds =
+      admin.listTableDescriptorsByNamespace(Bytes.toBytes(prefix + "ns1"));
     assertNotNull("Should have not returned null", htds);
-    assertEquals("Should have returned non-empty array", 1, htds.length);
+    assertEquals("Should have returned non-empty array", 1, htds.size());
 
     // get table descriptors for non-existing namespace
     runWithExpectedException(new Callable<Void>() {
       @Override
       public Void call() throws Exception {
-        admin.listTableDescriptorsByNamespace("non_existant_namespace");
+        admin.listTableDescriptorsByNamespace(Bytes.toBytes("non_existant_namespace"));
         return null;
       }
     }, NamespaceNotFoundException.class);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java
index cc62cbb..bf47e5b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java
@@ -118,7 +118,7 @@ public class TestPartialResultsFromClientSide {
   public static void setUpBeforeClass() throws Exception {
     TEST_UTIL.getConfiguration().setLong(HConstants.HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD, timeout);
     TEST_UTIL.startMiniCluster(MINICLUSTER_SIZE);
-    TEST_UTIL.getAdmin().setBalancerRunning(false, true);
+    TEST_UTIL.getAdmin().balancerSwitch(false, true);
     TABLE = createTestTable(TABLE_NAME, ROWS, FAMILIES, QUALIFIERS, VALUE);
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionLoad.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionLoad.java
index a390aca..0241a4c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionLoad.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionLoad.java
@@ -31,6 +31,7 @@ import java.util.concurrent.TimeUnit;
 import java.util.stream.Collectors;
 import org.apache.hadoop.hbase.ClusterMetrics.Option;
 import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
@@ -70,7 +71,7 @@ public class TestRegionLoad {
     UTIL.getConfiguration().setInt("hbase.regionserver.msginterval", MSG_INTERVAL);
     UTIL.startMiniCluster(4);
     admin = UTIL.getAdmin();
-    admin.setBalancerRunning(false, true);
+    admin.balancerSwitch(false, true);
     createTables();
   }
 
@@ -95,7 +96,7 @@ public class TestRegionLoad {
     // Check if regions match with the regionLoad from the server
     for (ServerName serverName : admin
         .getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)).getLiveServerMetrics().keySet()) {
-      List<HRegionInfo> regions = admin.getOnlineRegions(serverName);
+      List<RegionInfo> regions = admin.getRegions(serverName);
       LOG.info("serverName=" + serverName + ", regions=" +
           regions.stream().map(r -> r.getRegionNameAsString()).collect(Collectors.toList()));
       Collection<RegionLoad> regionLoads = admin.getRegionMetrics(serverName)
@@ -108,7 +109,7 @@ public class TestRegionLoad {
 
     // Check if regionLoad matches the table's regions and nothing is missed
     for (TableName table : new TableName[]{TABLE_1, TABLE_2, TABLE_3}) {
-      List<HRegionInfo> tableRegions = admin.getTableRegions(table);
+      List<RegionInfo> tableRegions = admin.getRegions(table);
 
       List<RegionLoad> regionLoads = Lists.newArrayList();
       for (ServerName serverName : admin
@@ -159,23 +160,21 @@ public class TestRegionLoad {
     assertEquals("regionLoads from SN should be empty", 0, regionLoads.size());
   }
 
-  private void checkRegionsAndRegionLoads(Collection<HRegionInfo> regions,
+  private void checkRegionsAndRegionLoads(Collection<RegionInfo> regions,
       Collection<RegionLoad> regionLoads) {
-
     for (RegionLoad load : regionLoads) {
       assertNotNull(load.regionLoadPB);
     }
 
-    assertEquals("No of regions and regionloads doesn't match",
-        regions.size(), regionLoads.size());
+    assertEquals("No of regions and regionloads doesn't match", regions.size(), regionLoads.size());
 
     Map<byte[], RegionLoad> regionLoadMap = Maps.newTreeMap(Bytes.BYTES_COMPARATOR);
     for (RegionLoad regionLoad : regionLoads) {
       regionLoadMap.put(regionLoad.getName(), regionLoad);
     }
-    for (HRegionInfo info : regions) {
+    for (RegionInfo info : regions) {
       assertTrue("Region not in regionLoadMap region:" + info.getRegionNameAsString() +
-          " regionMap: " + regionLoadMap, regionLoadMap.containsKey(info.getRegionName()));
+        " regionMap: " + regionLoadMap, regionLoadMap.containsKey(info.getRegionName()));
     }
   }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/CloneSnapshotFromClientNormalTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/CloneSnapshotFromClientNormalTestBase.java
index 3e26076..cb3ecd5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/CloneSnapshotFromClientNormalTestBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/CloneSnapshotFromClientNormalTestBase.java
@@ -34,7 +34,7 @@ public class CloneSnapshotFromClientNormalTestBase extends CloneSnapshotFromClie
     testCloneSnapshot(clonedTableName, emptySnapshot, 0);
   }
 
-  private void testCloneSnapshot(TableName tableName, byte[] snapshotName, int snapshotRows)
+  private void testCloneSnapshot(TableName tableName, String snapshotName, int snapshotRows)
       throws IOException, InterruptedException {
     // create a new table from snapshot
     admin.cloneSnapshot(snapshotName, tableName);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/CloneSnapshotFromClientTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/CloneSnapshotFromClientTestBase.java
index c74ec1d..00cc1a0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/CloneSnapshotFromClientTestBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/CloneSnapshotFromClientTestBase.java
@@ -40,10 +40,10 @@ public class CloneSnapshotFromClientTestBase {
 
   protected final byte[] FAMILY = Bytes.toBytes("cf");
 
-  protected byte[] emptySnapshot;
-  protected byte[] snapshotName0;
-  protected byte[] snapshotName1;
-  protected byte[] snapshotName2;
+  protected String emptySnapshot;
+  protected String snapshotName0;
+  protected String snapshotName1;
+  protected String snapshotName2;
   protected TableName tableName;
   protected int snapshot0Rows;
   protected int snapshot1Rows;
@@ -87,10 +87,10 @@ public class CloneSnapshotFromClientTestBase {
 
     long tid = System.currentTimeMillis();
     tableName = TableName.valueOf(getValidMethodName() + tid);
-    emptySnapshot = Bytes.toBytes("emptySnaptb-" + tid);
-    snapshotName0 = Bytes.toBytes("snaptb0-" + tid);
-    snapshotName1 = Bytes.toBytes("snaptb1-" + tid);
-    snapshotName2 = Bytes.toBytes("snaptb2-" + tid);
+    emptySnapshot = "emptySnaptb-" + tid;
+    snapshotName0 = "snaptb0-" + tid;
+    snapshotName1 = "snaptb1-" + tid;
+    snapshotName2 = "snaptb2-" + tid;
 
     createTableAndSnapshots();
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RestoreSnapshotFromClientSimpleTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RestoreSnapshotFromClientSimpleTestBase.java
index f4f2698..729679c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RestoreSnapshotFromClientSimpleTestBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RestoreSnapshotFromClientSimpleTestBase.java
@@ -24,7 +24,6 @@ import java.io.IOException;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.snapshot.CorruptedSnapshotException;
 import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
-import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.Test;
 
 public class RestoreSnapshotFromClientSimpleTestBase extends RestoreSnapshotFromClientTestBase {
@@ -63,7 +62,7 @@ public class RestoreSnapshotFromClientSimpleTestBase extends RestoreSnapshotFrom
 
   @Test
   public void testCorruptedSnapshot() throws IOException, InterruptedException {
-    SnapshotTestingUtils.corruptSnapshot(TEST_UTIL, Bytes.toString(snapshotName0));
+    SnapshotTestingUtils.corruptSnapshot(TEST_UTIL, snapshotName0);
     TableName cloneName =
       TableName.valueOf(getValidMethodName() + "-" + System.currentTimeMillis());
     try {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RestoreSnapshotFromClientTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RestoreSnapshotFromClientTestBase.java
index 047640f..666d67d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RestoreSnapshotFromClientTestBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RestoreSnapshotFromClientTestBase.java
@@ -42,10 +42,10 @@ public class RestoreSnapshotFromClientTestBase {
   protected final byte[] TEST_FAMILY2 = Bytes.toBytes("cf2");
 
   protected TableName tableName;
-  protected byte[] emptySnapshot;
-  protected byte[] snapshotName0;
-  protected byte[] snapshotName1;
-  protected byte[] snapshotName2;
+  protected String emptySnapshot;
+  protected String snapshotName0;
+  protected String snapshotName1;
+  protected String snapshotName2;
   protected int snapshot0Rows;
   protected int snapshot1Rows;
   protected Admin admin;
@@ -84,10 +84,10 @@ public class RestoreSnapshotFromClientTestBase {
 
     long tid = System.currentTimeMillis();
     tableName = TableName.valueOf(getValidMethodName() + "-" + tid);
-    emptySnapshot = Bytes.toBytes("emptySnaptb-" + tid);
-    snapshotName0 = Bytes.toBytes("snaptb0-" + tid);
-    snapshotName1 = Bytes.toBytes("snaptb1-" + tid);
-    snapshotName2 = Bytes.toBytes("snaptb2-" + tid);
+    emptySnapshot = "emptySnaptb-" + tid;
+    snapshotName0 = "snaptb0-" + tid;
+    snapshotName1 = "snaptb1-" + tid;
+    snapshotName2 = "snaptb2-" + tid;
 
     // create Table and disable it
     createTable();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
index 376aa92..d86be09 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
@@ -31,6 +31,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
+import java.util.regex.Pattern;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
@@ -59,7 +60,6 @@ import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.Threads;
 import org.junit.After;
 import org.junit.AfterClass;
-import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.Rule;
@@ -86,7 +86,7 @@ public class TestAdmin1 {
 
   private static final Logger LOG = LoggerFactory.getLogger(TestAdmin1.class);
   private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-  private Admin admin;
+  private static Admin ADMIN;
 
   @Rule
   public TestName name = new TestName();
@@ -98,21 +98,16 @@ public class TestAdmin1 {
     TEST_UTIL.getConfiguration().setInt("hbase.client.retries.number", 6);
     TEST_UTIL.getConfiguration().setBoolean("hbase.master.enabletable.roundrobin", true);
     TEST_UTIL.startMiniCluster(3);
+    ADMIN = TEST_UTIL.getAdmin();
   }
 
   @AfterClass
   public static void tearDownAfterClass() throws Exception {
     TEST_UTIL.shutdownMiniCluster();
   }
-
-  @Before
-  public void setUp() throws Exception {
-    this.admin = TEST_UTIL.getAdmin();
-  }
-
   @After
   public void tearDown() throws Exception {
-    for (HTableDescriptor htd : this.admin.listTables()) {
+    for (TableDescriptor htd : ADMIN.listTableDescriptors()) {
       TEST_UTIL.deleteTable(htd.getTableName());
     }
   }
@@ -122,7 +117,7 @@ public class TestAdmin1 {
     final TableName unknowntable = TableName.valueOf(name.getMethodName());
     Exception exception = null;
     try {
-      this.admin.compact(unknowntable);
+      ADMIN.compact(unknowntable);
     } catch (IOException e) {
       exception = e;
     }
@@ -130,7 +125,7 @@ public class TestAdmin1 {
 
     exception = null;
     try {
-      this.admin.flush(unknowntable);
+      ADMIN.flush(unknowntable);
     } catch (IOException e) {
       exception = e;
     }
@@ -138,7 +133,7 @@ public class TestAdmin1 {
 
     exception = null;
     try {
-      this.admin.split(unknowntable);
+      ADMIN.split(unknowntable);
     } catch (IOException e) {
       exception = e;
     }
@@ -153,7 +148,7 @@ public class TestAdmin1 {
     HColumnDescriptor nonexistentHcd = new HColumnDescriptor(nonexistentColumn);
     Exception exception = null;
     try {
-      this.admin.addColumnFamily(nonexistentTable, nonexistentHcd);
+      ADMIN.addColumnFamily(nonexistentTable, nonexistentHcd);
     } catch (IOException e) {
       exception = e;
     }
@@ -161,7 +156,7 @@ public class TestAdmin1 {
 
     exception = null;
     try {
-      this.admin.deleteTable(nonexistentTable);
+      ADMIN.deleteTable(nonexistentTable);
     } catch (IOException e) {
       exception = e;
     }
@@ -169,7 +164,7 @@ public class TestAdmin1 {
 
     exception = null;
     try {
-      this.admin.deleteColumnFamily(nonexistentTable, nonexistentColumn);
+      ADMIN.deleteColumnFamily(nonexistentTable, nonexistentColumn);
     } catch (IOException e) {
       exception = e;
     }
@@ -177,7 +172,7 @@ public class TestAdmin1 {
 
     exception = null;
     try {
-      this.admin.disableTable(nonexistentTable);
+      ADMIN.disableTable(nonexistentTable);
     } catch (IOException e) {
       exception = e;
     }
@@ -185,7 +180,7 @@ public class TestAdmin1 {
 
     exception = null;
     try {
-      this.admin.enableTable(nonexistentTable);
+      ADMIN.enableTable(nonexistentTable);
     } catch (IOException e) {
       exception = e;
     }
@@ -193,7 +188,7 @@ public class TestAdmin1 {
 
     exception = null;
     try {
-      this.admin.modifyColumnFamily(nonexistentTable, nonexistentHcd);
+      ADMIN.modifyColumnFamily(nonexistentTable, nonexistentHcd);
     } catch (IOException e) {
       exception = e;
     }
@@ -203,7 +198,7 @@ public class TestAdmin1 {
     try {
       HTableDescriptor htd = new HTableDescriptor(nonexistentTable);
       htd.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
-      this.admin.modifyTable(htd.getTableName(), htd);
+      ADMIN.modifyTable(htd);
     } catch (IOException e) {
       exception = e;
     }
@@ -214,11 +209,11 @@ public class TestAdmin1 {
     final TableName tableName = TableName.valueOf(name.getMethodName() + System.currentTimeMillis());
     HTableDescriptor htd = new HTableDescriptor(tableName);
     htd.addFamily(new HColumnDescriptor("cf"));
-    this.admin.createTable(htd);
+    ADMIN.createTable(htd);
     try {
       exception = null;
       try {
-        this.admin.deleteColumnFamily(htd.getTableName(), nonexistentHcd.getName());
+        ADMIN.deleteColumnFamily(htd.getTableName(), nonexistentHcd.getName());
       } catch (IOException e) {
         exception = e;
       }
@@ -227,15 +222,15 @@ public class TestAdmin1 {
 
       exception = null;
       try {
-        this.admin.modifyColumnFamily(htd.getTableName(), nonexistentHcd);
+        ADMIN.modifyColumnFamily(htd.getTableName(), nonexistentHcd);
       } catch (IOException e) {
         exception = e;
       }
       assertTrue("found=" + exception.getClass().getName(),
           exception instanceof InvalidFamilyOperationException);
     } finally {
-      this.admin.disableTable(tableName);
-      this.admin.deleteTable(tableName);
+      ADMIN.disableTable(tableName);
+      ADMIN.deleteTable(tableName);
     }
   }
 
@@ -253,7 +248,7 @@ public class TestAdmin1 {
     get.addColumn(HConstants.CATALOG_FAMILY, qualifier);
     ht.get(get);
 
-    this.admin.disableTable(ht.getName());
+    ADMIN.disableTable(ht.getName());
     assertTrue("Table must be disabled.", TEST_UTIL.getHBaseCluster()
         .getMaster().getTableStateManager().isTableState(
             ht.getName(), TableState.State.DISABLED));
@@ -281,7 +276,7 @@ public class TestAdmin1 {
       ok = true;
     }
     assertTrue(ok);
-    this.admin.enableTable(table);
+    ADMIN.enableTable(table);
     assertTrue("Table must be enabled.", TEST_UTIL.getHBaseCluster()
         .getMaster().getTableStateManager().isTableState(
             ht.getName(), TableState.State.ENABLED));
@@ -322,7 +317,10 @@ public class TestAdmin1 {
     ht1.get(get);
     ht2.get(get);
 
-    this.admin.disableTables("testDisableAndEnableTable.*");
+    TableName[] tableNames = ADMIN.listTableNames(Pattern.compile("testDisableAndEnableTable.*"));
+    for (TableName tableName: tableNames) {
+      ADMIN.disableTable(tableName);
+    }
 
     // Test that tables are disabled
     get = new Get(row);
@@ -340,7 +338,9 @@ public class TestAdmin1 {
 
 
     assertTrue(ok);
-    this.admin.enableTables("testDisableAndEnableTable.*");
+    for (TableName tableName: tableNames) {
+      ADMIN.enableTable(tableName);
+    }
 
     // Test that tables are enabled
     try {
@@ -364,12 +364,12 @@ public class TestAdmin1 {
 
   @Test
   public void testCreateTable() throws IOException {
-    HTableDescriptor [] tables = admin.listTables();
-    int numTables = tables.length;
+    List<TableDescriptor> tables = ADMIN.listTableDescriptors();
+    int numTables = tables.size();
     final TableName tableName = TableName.valueOf(name.getMethodName());
     TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY).close();
-    tables = this.admin.listTables();
-    assertEquals(numTables + 1, tables.length);
+    tables = ADMIN.listTableDescriptors();
+    assertEquals(numTables + 1, tables.size());
     assertTrue("Table must be enabled.",
         TEST_UTIL.getHBaseCluster().getMaster().getTableStateManager()
             .isTableState(tableName, TableState.State.ENABLED));
@@ -403,8 +403,8 @@ public class TestAdmin1 {
     assertEquals(3, TEST_UTIL.getHBaseCluster().getRegions(tableName).size());
 
     // Truncate & Verify
-    this.admin.disableTable(tableName);
-    this.admin.truncateTable(tableName, preserveSplits);
+    ADMIN.disableTable(tableName);
+    ADMIN.truncateTable(tableName, preserveSplits);
     table = TEST_UTIL.getConnection().getTable(tableName);
     try {
       assertEquals(0, TEST_UTIL.countRows(table));
@@ -427,7 +427,7 @@ public class TestAdmin1 {
     htd.addFamily(fam1);
     htd.addFamily(fam2);
     htd.addFamily(fam3);
-    this.admin.createTable(htd);
+    ADMIN.createTable(htd);
     Table table = TEST_UTIL.getConnection().getTable(htd.getTableName());
     TableDescriptor confirmedHtd = table.getDescriptor();
     assertEquals(0, TableDescriptor.COMPARATOR.compare(htd, confirmedHtd));
@@ -441,19 +441,19 @@ public class TestAdmin1 {
     final TableName tableName = TableName.valueOf(name.getMethodName());
     HTableDescriptor htd = new HTableDescriptor(tableName);
     htd.addFamily(fam1);
-    this.admin.createTable(htd);
+    ADMIN.createTable(htd);
     Table table = TEST_UTIL.getConnection().getTable(htd.getTableName());
-    long ts = this.admin.getLastMajorCompactionTimestamp(tableName);
+    long ts = ADMIN.getLastMajorCompactionTimestamp(tableName);
     assertEquals(0, ts);
     Put p = new Put(Bytes.toBytes("row1"));
     p.addColumn(Bytes.toBytes("fam1"), Bytes.toBytes("fam1"), Bytes.toBytes("fam1"));
     table.put(p);
-    ts = this.admin.getLastMajorCompactionTimestamp(tableName);
+    ts = ADMIN.getLastMajorCompactionTimestamp(tableName);
     // no files written -> no data
     assertEquals(0, ts);
 
-    this.admin.flush(tableName);
-    ts = this.admin.getLastMajorCompactionTimestamp(tableName);
+    ADMIN.flush(tableName);
+    ts = ADMIN.getLastMajorCompactionTimestamp(tableName);
     // still 0, we flushed a file, but no major compaction happened
     assertEquals(0, ts);
 
@@ -461,30 +461,30 @@ public class TestAdmin1 {
     try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
       regionName = l.getAllRegionLocations().get(0).getRegionInfo().getRegionName();
     }
-    long ts1 = this.admin.getLastMajorCompactionTimestampForRegion(regionName);
+    long ts1 = ADMIN.getLastMajorCompactionTimestampForRegion(regionName);
     assertEquals(ts, ts1);
     p = new Put(Bytes.toBytes("row2"));
     p.addColumn(Bytes.toBytes("fam1"), Bytes.toBytes("fam1"), Bytes.toBytes("fam1"));
     table.put(p);
-    this.admin.flush(tableName);
-    ts = this.admin.getLastMajorCompactionTimestamp(tableName);
+    ADMIN.flush(tableName);
+    ts = ADMIN.getLastMajorCompactionTimestamp(tableName);
     // make sure the region API returns the same value, as the old file is still around
     assertEquals(ts1, ts);
 
     TEST_UTIL.compact(tableName, true);
     table.put(p);
     // forces a wait for the compaction
-    this.admin.flush(tableName);
-    ts = this.admin.getLastMajorCompactionTimestamp(tableName);
+    ADMIN.flush(tableName);
+    ts = ADMIN.getLastMajorCompactionTimestamp(tableName);
     // after a compaction our earliest timestamp will have progressed forward
     assertTrue(ts > ts1);
 
     // region api still the same
-    ts1 = this.admin.getLastMajorCompactionTimestampForRegion(regionName);
+    ts1 = ADMIN.getLastMajorCompactionTimestampForRegion(regionName);
     assertEquals(ts, ts1);
     table.put(p);
-    this.admin.flush(tableName);
-    ts = this.admin.getLastMajorCompactionTimestamp(tableName);
+    ADMIN.flush(tableName);
+    ts = ADMIN.getLastMajorCompactionTimestamp(tableName);
     assertEquals(ts, ts1);
     table.close();
   }
@@ -509,18 +509,18 @@ public class TestAdmin1 {
     TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY).close();
 
     // Make table read only
-    TableDescriptor htd = TableDescriptorBuilder.newBuilder(this.admin.getDescriptor(tableName))
+    TableDescriptor htd = TableDescriptorBuilder.newBuilder(ADMIN.getDescriptor(tableName))
       .setReadOnly(true).build();
-    admin.modifyTable(htd);
+    ADMIN.modifyTable(htd);
 
     // try to modify the read only table now
-    htd = TableDescriptorBuilder.newBuilder(this.admin.getDescriptor(tableName))
+    htd = TableDescriptorBuilder.newBuilder(ADMIN.getDescriptor(tableName))
       .setCompactionEnabled(false).build();
-    admin.modifyTable(htd);
+    ADMIN.modifyTable(htd);
     // Delete the table
-    this.admin.disableTable(tableName);
-    this.admin.deleteTable(tableName);
-    assertFalse(this.admin.tableExists(tableName));
+    ADMIN.disableTable(tableName);
+    ADMIN.deleteTable(tableName);
+    assertFalse(ADMIN.tableExists(tableName));
   }
 
   @Test(expected = TableNotDisabledException.class)
@@ -529,17 +529,17 @@ public class TestAdmin1 {
     TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY).close();
 
     // Modify region replication count
-    TableDescriptor htd = TableDescriptorBuilder.newBuilder(admin.getDescriptor(tableName))
+    TableDescriptor htd = TableDescriptorBuilder.newBuilder(ADMIN.getDescriptor(tableName))
         .setRegionReplication(3).build();
     try {
       // try to modify the region replication count without disabling the table
-      admin.modifyTable(htd);
+      ADMIN.modifyTable(htd);
       fail("Expected an exception");
     } finally {
       // Delete the table
-      admin.disableTable(tableName);
-      admin.deleteTable(tableName);
-      assertFalse(admin.tableExists(tableName));
+      ADMIN.disableTable(tableName);
+      ADMIN.deleteTable(tableName);
+      assertFalse(ADMIN.tableExists(tableName));
     }
   }
 
@@ -549,14 +549,14 @@ public class TestAdmin1 {
   @Test
   public void testOnlineChangeTableSchema() throws IOException, InterruptedException {
     final TableName tableName = TableName.valueOf(name.getMethodName());
-    HTableDescriptor [] tables = admin.listTables();
-    int numTables = tables.length;
+    List<TableDescriptor> tables = ADMIN.listTableDescriptors();
+    int numTables = tables.size();
     TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY).close();
-    tables = this.admin.listTables();
-    assertEquals(numTables + 1, tables.length);
+    tables = ADMIN.listTableDescriptors();
+    assertEquals(numTables + 1, tables.size());
 
     // FIRST, do htabledescriptor changes.
-    HTableDescriptor htd = this.admin.getTableDescriptor(tableName);
+    HTableDescriptor htd = new HTableDescriptor(ADMIN.getDescriptor(tableName));
     // Make a copy and assert copy is good.
     HTableDescriptor copy = new HTableDescriptor(htd);
     assertTrue(htd.equals(copy));
@@ -571,12 +571,12 @@ public class TestAdmin1 {
     copy.setValue(key, key);
     boolean expectedException = false;
     try {
-      admin.modifyTable(tableName, copy);
+      ADMIN.modifyTable(copy);
     } catch (TableNotDisabledException re) {
       expectedException = true;
     }
     assertFalse(expectedException);
-    HTableDescriptor modifiedHtd = new HTableDescriptor(this.admin.getTableDescriptor(tableName));
+    HTableDescriptor modifiedHtd = new HTableDescriptor(ADMIN.getDescriptor(tableName));
     assertFalse(htd.equals(modifiedHtd));
     assertTrue(copy.equals(modifiedHtd));
     assertEquals(newFlushSize, modifiedHtd.getMemStoreFlushSize());
@@ -592,44 +592,44 @@ public class TestAdmin1 {
     final byte [] hcdName = hcd.getName();
     expectedException = false;
     try {
-      this.admin.modifyColumnFamily(tableName, hcd);
+      ADMIN.modifyColumnFamily(tableName, hcd);
     } catch (TableNotDisabledException re) {
       expectedException = true;
     }
     assertFalse(expectedException);
-    modifiedHtd = this.admin.getTableDescriptor(tableName);
+    modifiedHtd = new HTableDescriptor(ADMIN.getDescriptor(tableName));
     HColumnDescriptor modifiedHcd = modifiedHtd.getFamily(hcdName);
     assertEquals(newMaxVersions, modifiedHcd.getMaxVersions());
 
     // Try adding a column
-    assertFalse(this.admin.isTableDisabled(tableName));
+    assertFalse(ADMIN.isTableDisabled(tableName));
     final String xtracolName = "xtracol";
     HColumnDescriptor xtracol = new HColumnDescriptor(xtracolName);
     xtracol.setValue(xtracolName, xtracolName);
     expectedException = false;
     try {
-      this.admin.addColumnFamily(tableName, xtracol);
+      ADMIN.addColumnFamily(tableName, xtracol);
     } catch (TableNotDisabledException re) {
       expectedException = true;
     }
     // Add column should work even if the table is enabled
     assertFalse(expectedException);
-    modifiedHtd = this.admin.getTableDescriptor(tableName);
+    modifiedHtd = new HTableDescriptor(ADMIN.getDescriptor(tableName));
     hcd = modifiedHtd.getFamily(xtracol.getName());
     assertTrue(hcd != null);
     assertTrue(hcd.getValue(xtracolName).equals(xtracolName));
 
     // Delete the just-added column.
-    this.admin.deleteColumnFamily(tableName, xtracol.getName());
-    modifiedHtd = this.admin.getTableDescriptor(tableName);
+    ADMIN.deleteColumnFamily(tableName, xtracol.getName());
+    modifiedHtd = new HTableDescriptor(ADMIN.getDescriptor(tableName));
     hcd = modifiedHtd.getFamily(xtracol.getName());
     assertTrue(hcd == null);
 
     // Delete the table
-    this.admin.disableTable(tableName);
-    this.admin.deleteTable(tableName);
-    this.admin.listTables();
-    assertFalse(this.admin.tableExists(tableName));
+    ADMIN.disableTable(tableName);
+    ADMIN.deleteTable(tableName);
+    ADMIN.listTableDescriptors();
+    assertFalse(ADMIN.tableExists(tableName));
   }
 
   protected void verifyRoundRobinDistribution(ClusterConnection c, RegionLocator regionLocator, int
@@ -667,7 +667,7 @@ public class TestAdmin1 {
     final TableName tableName = TableName.valueOf(name.getMethodName());
     HTableDescriptor desc = new HTableDescriptor(tableName);
     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
-    admin.createTable(desc);
+    ADMIN.createTable(desc);
     List<HRegionLocation> regions;
     try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
       regions = l.getAllRegionLocations();
@@ -677,7 +677,7 @@ public class TestAdmin1 {
     TableName TABLE_2 = TableName.valueOf(tableName.getNameAsString() + "_2");
     desc = new HTableDescriptor(TABLE_2);
     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
-    admin.createTable(desc, new byte[][]{new byte[]{42}});
+    ADMIN.createTable(desc, new byte[][]{new byte[]{42}});
     try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(TABLE_2)) {
       regions = l.getAllRegionLocations();
       assertEquals("Table should have only 2 region", 2, regions.size());
@@ -686,7 +686,7 @@ public class TestAdmin1 {
     TableName TABLE_3 = TableName.valueOf(tableName.getNameAsString() + "_3");
     desc = new HTableDescriptor(TABLE_3);
     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
-    admin.createTable(desc, Bytes.toBytes("a"), Bytes.toBytes("z"), 3);
+    ADMIN.createTable(desc, Bytes.toBytes("a"), Bytes.toBytes("z"), 3);
     try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(TABLE_3)) {
       regions = l.getAllRegionLocations();
       assertEquals("Table should have only 3 region", 3, regions.size());
@@ -696,7 +696,7 @@ public class TestAdmin1 {
     desc = new HTableDescriptor(TABLE_4);
     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
     try {
-      admin.createTable(desc, Bytes.toBytes("a"), Bytes.toBytes("z"), 2);
+      ADMIN.createTable(desc, Bytes.toBytes("a"), Bytes.toBytes("z"), 2);
       fail("Should not be able to create a table with only 2 regions using this API.");
     } catch (IllegalArgumentException eae) {
     // Expected
@@ -705,7 +705,7 @@ public class TestAdmin1 {
     TableName TABLE_5 = TableName.valueOf(tableName.getNameAsString() + "_5");
     desc = new HTableDescriptor(TABLE_5);
     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
-    admin.createTable(desc, new byte[] { 1 }, new byte[] { 127 }, 16);
+    ADMIN.createTable(desc, new byte[] { 1 }, new byte[] { 127 }, 16);
     try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(TABLE_5)) {
       regions = l.getAllRegionLocations();
       assertEquals("Table should have 16 region", 16, regions.size());
@@ -715,7 +715,6 @@ public class TestAdmin1 {
   @Test
   public void testCreateTableWithRegions() throws IOException, InterruptedException {
     final TableName tableName = TableName.valueOf(name.getMethodName());
-
     byte [][] splitKeys = {
         new byte [] { 1, 1, 1 },
         new byte [] { 2, 2, 2 },
@@ -731,9 +730,9 @@ public class TestAdmin1 {
 
     HTableDescriptor desc = new HTableDescriptor(tableName);
     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
-    admin.createTable(desc, splitKeys);
+    ADMIN.createTable(desc, splitKeys);
 
-    boolean tableAvailable = admin.isTableAvailable(tableName, splitKeys);
+    boolean tableAvailable = ADMIN.isTableAvailable(tableName);
     assertTrue("Table should be created with splitKyes + 1 rows in META", tableAvailable);
 
     List<HRegionLocation> regions;
@@ -799,8 +798,7 @@ public class TestAdmin1 {
 
     desc = new HTableDescriptor(TABLE_2);
     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
-    admin = TEST_UTIL.getAdmin();
-    admin.createTable(desc, startKey, endKey, expectedRegions);
+    ADMIN.createTable(desc, startKey, endKey, expectedRegions);
 
     try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(TABLE_2)) {
       regions = l.getAllRegionLocations();
@@ -854,8 +852,7 @@ public class TestAdmin1 {
 
     desc = new HTableDescriptor(TABLE_3);
     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
-    admin = TEST_UTIL.getAdmin();
-    admin.createTable(desc, startKey, endKey, expectedRegions);
+    ADMIN.createTable(desc, startKey, endKey, expectedRegions);
 
 
     try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(TABLE_3)) {
@@ -880,7 +877,7 @@ public class TestAdmin1 {
     desc = new HTableDescriptor(TABLE_4);
     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
     try {
-      admin.createTable(desc, splitKeys);
+      ADMIN.createTable(desc, splitKeys);
       assertTrue("Should not be able to create this table because of " +
           "duplicate split keys", false);
     } catch(IllegalArgumentException iae) {
@@ -889,21 +886,6 @@ public class TestAdmin1 {
   }
 
   @Test
-  public void testTableAvailableWithRandomSplitKeys() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
-    HTableDescriptor desc = new HTableDescriptor(tableName);
-    desc.addFamily(new HColumnDescriptor("col"));
-    byte[][] splitKeys = new byte[1][];
-    splitKeys = new byte [][] {
-        new byte [] { 1, 1, 1 },
-        new byte [] { 2, 2, 2 }
-    };
-    admin.createTable(desc);
-    boolean tableAvailable = admin.isTableAvailable(tableName, splitKeys);
-    assertFalse("Table should be created with 1 row in META", tableAvailable);
-  }
-
-  @Test
   public void testCreateTableWithOnlyEmptyStartRow() throws IOException {
     final byte[] tableName = Bytes.toBytes(name.getMethodName());
     byte[][] splitKeys = new byte[1][];
@@ -911,7 +893,7 @@ public class TestAdmin1 {
     HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
     desc.addFamily(new HColumnDescriptor("col"));
     try {
-      admin.createTable(desc, splitKeys);
+      ADMIN.createTable(desc, splitKeys);
       fail("Test case should fail as empty split key is passed.");
     } catch (IllegalArgumentException e) {
     }
@@ -927,7 +909,7 @@ public class TestAdmin1 {
     HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
     desc.addFamily(new HColumnDescriptor("col"));
     try {
-      admin.createTable(desc, splitKeys);
+      ADMIN.createTable(desc, splitKeys);
       fail("Test case should fail as empty split key is passed.");
     } catch (IllegalArgumentException e) {
       LOG.info("Expected ", e);
@@ -938,10 +920,10 @@ public class TestAdmin1 {
   public void testTableExist() throws IOException {
     final TableName table = TableName.valueOf(name.getMethodName());
     boolean exist;
-    exist = this.admin.tableExists(table);
+    exist = ADMIN.tableExists(table);
     assertEquals(false, exist);
     TEST_UTIL.createTable(table, HConstants.CATALOG_FAMILY);
-    exist = this.admin.tableExists(table);
+    exist = ADMIN.tableExists(table);
     assertEquals(true, exist);
   }
 
@@ -979,7 +961,7 @@ public class TestAdmin1 {
     int expectedRegions = splitKeys.length + 1;
     HTableDescriptor desc = new HTableDescriptor(tableName);
     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
-    admin.createTable(desc, splitKeys);
+    ADMIN.createTable(desc, splitKeys);
 
     try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
       List<HRegionLocation> regions = l.getAllRegionLocations();
@@ -988,9 +970,9 @@ public class TestAdmin1 {
           "Tried to create " + expectedRegions + " regions " + "but only found " + regions.size(),
           expectedRegions, regions.size());
       // Disable table.
-      admin.disableTable(tableName);
+      ADMIN.disableTable(tableName);
       // Enable table, use retain assignment to assign regions.
-      admin.enableTable(tableName);
+      ADMIN.enableTable(tableName);
       List<HRegionLocation> regions2 = l.getAllRegionLocations();
 
       // Check the assignment.
@@ -1041,7 +1023,7 @@ public class TestAdmin1 {
     for (int i = 0; i < rowCounts.length; i++) {
       sb.append("_").append(Integer.toString(rowCounts[i]));
     }
-    assertFalse(admin.tableExists(tableName));
+    assertFalse(ADMIN.tableExists(tableName));
     try (final Table table = TEST_UTIL.createTable(tableName, familyNames,
       numVersions, blockSize);
       final RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
@@ -1089,7 +1071,7 @@ public class TestAdmin1 {
 
       // Split the table
       if (async) {
-        this.admin.split(tableName, splitPoint);
+        ADMIN.split(tableName, splitPoint);
         final AtomicInteger count = new AtomicInteger(0);
         Thread t = new Thread("CheckForSplit") {
           @Override public void run() {
@@ -1122,7 +1104,7 @@ public class TestAdmin1 {
         t.join();
       } else {
         // Sync split region, no need to create a thread to check
-        ((HBaseAdmin)admin).splitRegionSync(m.get(0).getRegionInfo().getRegionName(), splitPoint);
+        ADMIN.splitRegionAsync(m.get(0).getRegion().getRegionName(), splitPoint).get();
       }
 
       // Verify row count
@@ -1214,7 +1196,7 @@ public class TestAdmin1 {
     // the element at index 1 would be a replica (since the metareader gives us ordered
     // regions). Try splitting that region via the split API . Should fail
     try {
-      TEST_UTIL.getAdmin().splitRegion(regions.get(1).getFirst().getRegionName());
+      TEST_UTIL.getAdmin().splitRegionAsync(regions.get(1).getFirst().getRegionName(), null).get();
     } catch (IllegalArgumentException ex) {
       gotException = true;
     }
@@ -1287,54 +1269,54 @@ public class TestAdmin1 {
   public void testEnableDisableAddColumnDeleteColumn() throws Exception {
     final TableName tableName = TableName.valueOf(name.getMethodName());
     TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY).close();
-    while (!this.admin.isTableEnabled(TableName.valueOf(name.getMethodName()))) {
+    while (!ADMIN.isTableEnabled(TableName.valueOf(name.getMethodName()))) {
       Thread.sleep(10);
     }
-    this.admin.disableTable(tableName);
+    ADMIN.disableTable(tableName);
     try {
       TEST_UTIL.getConnection().getTable(tableName);
     } catch (org.apache.hadoop.hbase.DoNotRetryIOException e) {
       //expected
     }
 
-    this.admin.addColumnFamily(tableName, new HColumnDescriptor("col2"));
-    this.admin.enableTable(tableName);
+    ADMIN.addColumnFamily(tableName, new HColumnDescriptor("col2"));
+    ADMIN.enableTable(tableName);
     try {
-      this.admin.deleteColumnFamily(tableName, Bytes.toBytes("col2"));
+      ADMIN.deleteColumnFamily(tableName, Bytes.toBytes("col2"));
     } catch (TableNotDisabledException e) {
       LOG.info(e.toString(), e);
     }
-    this.admin.disableTable(tableName);
-    this.admin.deleteTable(tableName);
+    ADMIN.disableTable(tableName);
+    ADMIN.deleteTable(tableName);
   }
 
   @Test
   public void testDeleteLastColumnFamily() throws Exception {
     final TableName tableName = TableName.valueOf(name.getMethodName());
     TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY).close();
-    while (!this.admin.isTableEnabled(TableName.valueOf(name.getMethodName()))) {
+    while (!ADMIN.isTableEnabled(TableName.valueOf(name.getMethodName()))) {
       Thread.sleep(10);
     }
 
     // test for enabled table
     try {
-      this.admin.deleteColumnFamily(tableName, HConstants.CATALOG_FAMILY);
+      ADMIN.deleteColumnFamily(tableName, HConstants.CATALOG_FAMILY);
       fail("Should have failed to delete the only column family of a table");
     } catch (InvalidFamilyOperationException ex) {
       // expected
     }
 
     // test for disabled table
-    this.admin.disableTable(tableName);
+    ADMIN.disableTable(tableName);
 
     try {
-      this.admin.deleteColumnFamily(tableName, HConstants.CATALOG_FAMILY);
+      ADMIN.deleteColumnFamily(tableName, HConstants.CATALOG_FAMILY);
       fail("Should have failed to delete the only column family of a table");
     } catch (InvalidFamilyOperationException ex) {
       // expected
     }
 
-    this.admin.deleteTable(tableName);
+    ADMIN.deleteTable(tableName);
   }
 
   /*
@@ -1365,7 +1347,7 @@ public class TestAdmin1 {
     puts.add(p);
     try {
       table.put(puts);
-      admin.flush(tableName);
+      ADMIN.flush(tableName);
 
       List<HRegion> regions = TEST_UTIL.getMiniHBaseCluster().getRegions(tableName);
       for (HRegion r : regions) {
@@ -1385,9 +1367,9 @@ public class TestAdmin1 {
         }
       }
     } finally {
-      if (admin.isTableEnabled(tableName)) {
-        this.admin.disableTable(tableName);
-        this.admin.deleteTable(tableName);
+      if (ADMIN.isTableEnabled(tableName)) {
+        ADMIN.disableTable(tableName);
+        ADMIN.deleteTable(tableName);
       }
     }
   }
@@ -1410,29 +1392,29 @@ public class TestAdmin1 {
       RegionInfo regionB;
 
       // merge with full name
-      tableRegions = admin.getRegions(tableName);
-      assertEquals(3, admin.getTableRegions(tableName).size());
+      tableRegions = ADMIN.getRegions(tableName);
+      assertEquals(3, ADMIN.getRegions(tableName).size());
       regionA = tableRegions.get(0);
       regionB = tableRegions.get(1);
       // TODO convert this to version that is synchronous (See HBASE-16668)
-      admin.mergeRegionsAsync(regionA.getRegionName(), regionB.getRegionName(), false)
+      ADMIN.mergeRegionsAsync(regionA.getRegionName(), regionB.getRegionName(), false)
           .get(60, TimeUnit.SECONDS);
 
-      assertEquals(2, admin.getTableRegions(tableName).size());
+      assertEquals(2, ADMIN.getRegions(tableName).size());
 
       // merge with encoded name
-      tableRegions = admin.getRegions(tableName);
+      tableRegions = ADMIN.getRegions(tableName);
       regionA = tableRegions.get(0);
       regionB = tableRegions.get(1);
       // TODO convert this to version that is synchronous (See HBASE-16668)
-      admin.mergeRegionsAsync(
+      ADMIN.mergeRegionsAsync(
         regionA.getEncodedNameAsBytes(), regionB.getEncodedNameAsBytes(), false)
           .get(60, TimeUnit.SECONDS);
 
-      assertEquals(1, admin.getTableRegions(tableName).size());
+      assertEquals(1, ADMIN.getRegions(tableName).size());
     } finally {
-      this.admin.disableTable(tableName);
-      this.admin.deleteTable(tableName);
+      ADMIN.disableTable(tableName);
+      ADMIN.deleteTable(tableName);
     }
   }
 
@@ -1451,16 +1433,16 @@ public class TestAdmin1 {
       p.addColumn(Bytes.toBytes("f"), q1, v1);
       table.put(p);
     }
-    this.admin.flush(tableName);
+    ADMIN.flush(tableName);
     try {
-      this.admin.split(tableName, Bytes.toBytes("row5"));
+      ADMIN.split(tableName, Bytes.toBytes("row5"));
       Threads.sleep(10000);
     } catch (Exception e) {
       // Nothing to do.
     }
     // Split should not happen.
     List<RegionInfo> allRegions = MetaTableAccessor.getTableRegions(
-        this.admin.getConnection(), tableName, true);
+        ADMIN.getConnection(), tableName, true);
     assertEquals(1, allRegions.size());
   }
 
@@ -1502,15 +1484,15 @@ public class TestAdmin1 {
             .setTimeToLive(TTL)
             .build()
         ).build();
-    admin.createTable(tableDesc, splitKeys);
+    ADMIN.createTable(tableDesc, splitKeys);
 
     assertEquals(NUM_REGIONS, TEST_UTIL.getHBaseCluster().getRegions(tableName).size());
     assertTrue("Table should be created with splitKyes + 1 rows in META",
-        admin.isTableAvailable(tableName, splitKeys));
+      ADMIN.isTableAvailable(tableName));
 
     // clone & Verify
-    admin.cloneTableSchema(tableName, newTableName, preserveSplits);
-    TableDescriptor newTableDesc = admin.getDescriptor(newTableName);
+    ADMIN.cloneTableSchema(tableName, newTableName, preserveSplits);
+    TableDescriptor newTableDesc = ADMIN.getDescriptor(newTableName);
 
     assertEquals(NUM_FAMILYS, newTableDesc.getColumnFamilyCount());
     assertEquals(BLOCK_SIZE, newTableDesc.getColumnFamily(FAMILY_1).getBlocksize());
@@ -1521,7 +1503,7 @@ public class TestAdmin1 {
     if (preserveSplits) {
       assertEquals(NUM_REGIONS, TEST_UTIL.getHBaseCluster().getRegions(newTableName).size());
       assertTrue("New table should be created with splitKyes + 1 rows in META",
-          admin.isTableAvailable(newTableName, splitKeys));
+          ADMIN.isTableAvailable(newTableName));
     } else {
       assertEquals(1, TEST_UTIL.getHBaseCluster().getRegions(newTableName).size());
     }
@@ -1533,7 +1515,7 @@ public class TestAdmin1 {
     final TableName newTableName = TableName.valueOf(tableName.getNameAsString() + "_new");
     // test for non-existent source table
     try {
-      admin.cloneTableSchema(tableName, newTableName, false);
+      ADMIN.cloneTableSchema(tableName, newTableName, false);
       fail("Should have failed to create a new table by cloning non-existent source table.");
     } catch (TableNotFoundException ex) {
       // expected
@@ -1549,7 +1531,7 @@ public class TestAdmin1 {
     TEST_UTIL.createTable(newTableName, FAMILY_0);
     // test for existent destination table
     try {
-      admin.cloneTableSchema(tableName, newTableName, false);
+      ADMIN.cloneTableSchema(tableName, newTableName, false);
       fail("Should have failed to create a existent table.");
     } catch (TableExistsException ex) {
       // expected
@@ -1564,15 +1546,15 @@ public class TestAdmin1 {
         .setRegionReplication(5)
         .build();
 
-    admin.createTable(desc);
+    ADMIN.createTable(desc);
 
     int maxFileSize = 10000000;
     TableDescriptor newDesc = TableDescriptorBuilder.newBuilder(desc)
         .setMaxFileSize(maxFileSize)
         .build();
 
-    admin.modifyTable(newDesc);
-    TableDescriptor newTableDesc = admin.getDescriptor(tableName);
+    ADMIN.modifyTable(newDesc);
+    TableDescriptor newTableDesc = ADMIN.getDescriptor(tableName);
     assertEquals(maxFileSize, newTableDesc.getMaxFileSize());
   }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
index 2c14eaf..b928750 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
@@ -63,7 +63,6 @@ import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Assert;
-import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.Rule;
@@ -89,7 +88,7 @@ public class TestAdmin2 {
 
   private static final Logger LOG = LoggerFactory.getLogger(TestAdmin2.class);
   private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-  private Admin admin;
+  private static Admin ADMIN;
 
   @Rule
   public TestName name = new TestName();
@@ -103,6 +102,7 @@ public class TestAdmin2 {
     TEST_UTIL.getConfiguration().setInt(HConstants.REGION_SERVER_HANDLER_COUNT, 30);
     TEST_UTIL.getConfiguration().setBoolean("hbase.master.enabletable.roundrobin", true);
     TEST_UTIL.startMiniCluster(3);
+    ADMIN = TEST_UTIL.getAdmin();
   }
 
   @AfterClass
@@ -110,14 +110,9 @@ public class TestAdmin2 {
     TEST_UTIL.shutdownMiniCluster();
   }
 
-  @Before
-  public void setUp() throws Exception {
-    this.admin = TEST_UTIL.getHBaseAdmin();
-  }
-
   @After
   public void tearDown() throws Exception {
-    for (HTableDescriptor htd : this.admin.listTables()) {
+    for (TableDescriptor htd : ADMIN.listTableDescriptors()) {
       TEST_UTIL.deleteTable(htd.getTableName());
     }
   }
@@ -126,7 +121,7 @@ public class TestAdmin2 {
   public void testCreateBadTables() throws IOException {
     String msg = null;
     try {
-      this.admin.createTable(new HTableDescriptor(TableName.META_TABLE_NAME));
+      ADMIN.createTable(new HTableDescriptor(TableName.META_TABLE_NAME));
     } catch(TableExistsException e) {
       msg = e.toString();
     }
@@ -141,7 +136,7 @@ public class TestAdmin2 {
     Thread [] threads = new Thread [count];
     final AtomicInteger successes = new AtomicInteger(0);
     final AtomicInteger failures = new AtomicInteger(0);
-    final Admin localAdmin = this.admin;
+    final Admin localAdmin = ADMIN;
     for (int i = 0; i < count; i++) {
       threads[i] = new Thread(Integer.toString(i)) {
         @Override
@@ -186,8 +181,8 @@ public class TestAdmin2 {
     HTableDescriptor htd2 = new HTableDescriptor(TableName.valueOf(name));
     htd1.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
     htd2.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
-    admin.createTable(htd1);
-    admin.createTable(htd2);
+    ADMIN.createTable(htd1);
+    ADMIN.createTable(htd2);
     // Before fix, below would fail throwing a NoServerForRegionException.
     TEST_UTIL.getConnection().getTable(htd2.getTableName()).close();
   }
@@ -197,7 +192,6 @@ public class TestAdmin2 {
    * Thus creating of table with lots of regions can cause RPC timeout
    * After the fix to make createTable truly async, RPC timeout shouldn't be an
    * issue anymore
-   * @throws Exception
    */
   @Test
   public void testCreateTableRPCTimeOut() throws Exception {
@@ -238,7 +232,6 @@ public class TestAdmin2 {
   /**
    * Test that user table names can contain '-' and '.' so long as they do not
    * start with same. HBASE-771
-   * @throws IOException
    */
   @Test
   public void testTableNames() throws IOException {
@@ -267,7 +260,6 @@ public class TestAdmin2 {
 
   /**
    * For HADOOP-2579
-   * @throws IOException
    */
   @Test (expected=TableExistsException.class)
   public void testTableExistsExceptionWithATable() throws IOException {
@@ -278,41 +270,35 @@ public class TestAdmin2 {
 
   /**
    * Can't disable a table if the table isn't in enabled state
-   * @throws IOException
    */
   @Test (expected=TableNotEnabledException.class)
   public void testTableNotEnabledExceptionWithATable() throws IOException {
     final TableName name = TableName.valueOf(this.name.getMethodName());
     TEST_UTIL.createTable(name, HConstants.CATALOG_FAMILY).close();
-    this.admin.disableTable(name);
-    this.admin.disableTable(name);
+    ADMIN.disableTable(name);
+    ADMIN.disableTable(name);
   }
 
   /**
    * Can't enable a table if the table isn't in disabled state
-   * @throws IOException
    */
   @Test (expected=TableNotDisabledException.class)
   public void testTableNotDisabledExceptionWithATable() throws IOException {
     final TableName name = TableName.valueOf(this.name.getMethodName());
-    Table t = TEST_UTIL.createTable(name, HConstants.CATALOG_FAMILY);
-    try {
-    this.admin.enableTable(name);
-    }finally {
-       t.close();
+    try (Table t = TEST_UTIL.createTable(name, HConstants.CATALOG_FAMILY)) {
+      ADMIN.enableTable(name);
     }
   }
 
   /**
    * For HADOOP-2579
-   * @throws IOException
    */
-  @Test (expected=TableNotFoundException.class)
+  @Test(expected = TableNotFoundException.class)
   public void testTableNotFoundExceptionWithoutAnyTables() throws IOException {
-    TableName tableName = TableName
-        .valueOf("testTableNotFoundExceptionWithoutAnyTables");
-    Table ht = TEST_UTIL.getConnection().getTable(tableName);
-    ht.get(new Get(Bytes.toBytes("e")));
+    TableName tableName = TableName.valueOf("testTableNotFoundExceptionWithoutAnyTables");
+    try (Table ht = TEST_UTIL.getConnection().getTable(tableName)) {
+      ht.get(new Get(Bytes.toBytes("e")));
+    }
   }
 
   @Test
@@ -326,7 +312,7 @@ public class TestAdmin2 {
     for (RegionInfo regionInfo : onlineRegions) {
       if (!regionInfo.getTable().isSystemTable()) {
         info = regionInfo;
-        admin.unassign(regionInfo.getRegionName(), true);
+        ADMIN.unassign(regionInfo.getRegionName(), true);
       }
     }
     boolean isInList = ProtobufUtil.getOnlineRegions(
@@ -356,7 +342,7 @@ public class TestAdmin2 {
         if (regionInfo.getRegionNameAsString().contains(name)) {
           info = regionInfo;
           try {
-            admin.unassign(Bytes.toBytes("sample"), true);
+            ADMIN.unassign(Bytes.toBytes("sample"), true);
           } catch (UnknownRegionException nsre) {
             // expected, ignore it
           }
@@ -380,7 +366,7 @@ public class TestAdmin2 {
       if (!regionInfo.isMetaRegion()) {
         if (regionInfo.getRegionNameAsString().contains("TestHBACloseRegion2")) {
           info = regionInfo;
-          admin.unassign(regionInfo.getRegionName(), true);
+          ADMIN.unassign(regionInfo.getRegionName(), true);
         }
       }
     }
@@ -418,7 +404,7 @@ public class TestAdmin2 {
     HColumnDescriptor hcd = new HColumnDescriptor("value");
     htd.addFamily(hcd);
 
-    admin.createTable(htd, null);
+    ADMIN.createTable(htd, null);
   }
 
   /**
@@ -438,9 +424,9 @@ public class TestAdmin2 {
 
     HTableDescriptor desc = new HTableDescriptor(tableName);
     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
-    admin.createTable(desc, startKey, endKey, expectedRegions);
+    ADMIN.createTable(desc, startKey, endKey, expectedRegions);
 
-    List<RegionInfo> RegionInfos = admin.getRegions(tableName);
+    List<RegionInfo> RegionInfos = ADMIN.getRegions(tableName);
 
     assertEquals("Tried to create " + expectedRegions + " regions " +
         "but only found " + RegionInfos.size(),
@@ -479,7 +465,7 @@ public class TestAdmin2 {
     for (HRegion r : regionServer.getOnlineRegionsLocalContext()) {
       r.flush(true);
     }
-    admin.rollWALWriter(regionServer.getServerName());
+    ADMIN.rollWALWriter(regionServer.getServerName());
     int count = AbstractFSWALProvider.getNumRolledLogFiles(regionServer.getWAL(null));
     LOG.info("after flushing all regions and rolling logs there are " +
         count + " log files");
@@ -536,7 +522,7 @@ public class TestAdmin2 {
     // Create the test table and open it
     HTableDescriptor desc = new HTableDescriptor(tableName);
     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
-    admin.createTable(desc);
+    ADMIN.createTable(desc);
     Table table = TEST_UTIL.getConnection().getTable(tableName);
 
     HRegionServer regionServer = TEST_UTIL.getRSForFirstRegionInTable(tableName);
@@ -585,7 +571,7 @@ public class TestAdmin2 {
   @Test
   public void testDisableCatalogTable() throws Exception {
     try {
-      this.admin.disableTable(TableName.META_TABLE_NAME);
+      ADMIN.disableTable(TableName.META_TABLE_NAME);
       fail("Expected to throw ConstraintException");
     } catch (ConstraintException e) {
     }
@@ -601,13 +587,13 @@ public class TestAdmin2 {
   @Test
   public void testIsEnabledOrDisabledOnUnknownTable() throws Exception {
     try {
-      admin.isTableEnabled(TableName.valueOf(name.getMethodName()));
+      ADMIN.isTableEnabled(TableName.valueOf(name.getMethodName()));
       fail("Test should fail if isTableEnabled called on unknown table.");
     } catch (IOException e) {
     }
 
     try {
-      admin.isTableDisabled(TableName.valueOf(name.getMethodName()));
+      ADMIN.isTableDisabled(TableName.valueOf(name.getMethodName()));
       fail("Test should fail if isTableDisabled called on unknown table.");
     } catch (IOException e) {
     }
@@ -636,46 +622,46 @@ public class TestAdmin2 {
 
   @Test
   public void testBalancer() throws Exception {
-    boolean initialState = admin.isBalancerEnabled();
+    boolean initialState = ADMIN.isBalancerEnabled();
 
     // Start the balancer, wait for it.
-    boolean prevState = admin.setBalancerRunning(!initialState, true);
+    boolean prevState = ADMIN.balancerSwitch(!initialState, true);
 
     // The previous state should be the original state we observed
     assertEquals(initialState, prevState);
 
     // Current state should be opposite of the original
-    assertEquals(!initialState, admin.isBalancerEnabled());
+    assertEquals(!initialState, ADMIN.isBalancerEnabled());
 
     // Reset it back to what it was
-    prevState = admin.setBalancerRunning(initialState, true);
+    prevState = ADMIN.balancerSwitch(initialState, true);
 
     // The previous state should be the opposite of the initial state
     assertEquals(!initialState, prevState);
     // Current state should be the original state again
-    assertEquals(initialState, admin.isBalancerEnabled());
+    assertEquals(initialState, ADMIN.isBalancerEnabled());
   }
 
   @Test
   public void testRegionNormalizer() throws Exception {
-    boolean initialState = admin.isNormalizerEnabled();
+    boolean initialState = ADMIN.isNormalizerEnabled();
 
     // flip state
-    boolean prevState = admin.setNormalizerRunning(!initialState);
+    boolean prevState = ADMIN.normalizerSwitch(!initialState);
 
     // The previous state should be the original state we observed
     assertEquals(initialState, prevState);
 
     // Current state should be opposite of the original
-    assertEquals(!initialState, admin.isNormalizerEnabled());
+    assertEquals(!initialState, ADMIN.isNormalizerEnabled());
 
     // Reset it back to what it was
-    prevState = admin.setNormalizerRunning(initialState);
+    prevState = ADMIN.normalizerSwitch(initialState);
 
     // The previous state should be the opposite of the initial state
     assertEquals(!initialState, prevState);
     // Current state should be the original state again
-    assertEquals(initialState, admin.isNormalizerEnabled());
+    assertEquals(initialState, ADMIN.isNormalizerEnabled());
   }
 
   @Test
@@ -683,32 +669,32 @@ public class TestAdmin2 {
     Random randomGenerator = new Random();
     long procId = randomGenerator.nextLong();
 
-    boolean abortResult = admin.abortProcedure(procId, true);
+    boolean abortResult = ADMIN.abortProcedure(procId, true);
     assertFalse(abortResult);
   }
 
   @Test
   public void testGetProcedures() throws Exception {
-    String procList = admin.getProcedures();
+    String procList = ADMIN.getProcedures();
     assertTrue(procList.startsWith("["));
   }
 
   @Test
   public void testGetLocks() throws Exception {
-    String lockList = admin.getLocks();
+    String lockList = ADMIN.getLocks();
     assertTrue(lockList.startsWith("["));
   }
 
   @Test
   public void testDecommissionRegionServers() throws Exception {
-    List<ServerName> decommissionedRegionServers = admin.listDecommissionedRegionServers();
+    List<ServerName> decommissionedRegionServers = ADMIN.listDecommissionedRegionServers();
     assertTrue(decommissionedRegionServers.isEmpty());
 
     final TableName tableName = TableName.valueOf(name.getMethodName());
     TEST_UTIL.createMultiRegionTable(tableName, Bytes.toBytes("f"), 6);
 
     ArrayList<ServerName> clusterRegionServers =
-        new ArrayList<>(admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
+        new ArrayList<>(ADMIN.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
           .getLiveServerMetrics().keySet());
 
     assertEquals(3, clusterRegionServers.size());
@@ -718,8 +704,8 @@ public class TestAdmin2 {
     // leaving one online.
     int i;
     for (i = 0; i < clusterRegionServers.size(); i++) {
-      List<RegionInfo> regionsOnServer = admin.getRegions(clusterRegionServers.get(i));
-      if (admin.getRegions(clusterRegionServers.get(i)).stream().anyMatch(p -> p.isMetaRegion())) {
+      List<RegionInfo> regionsOnServer = ADMIN.getRegions(clusterRegionServers.get(i));
+      if (ADMIN.getRegions(clusterRegionServers.get(i)).stream().anyMatch(p -> p.isMetaRegion())) {
         serversToDecommssion.put(clusterRegionServers.get(i), regionsOnServer);
         break;
       }
@@ -728,13 +714,13 @@ public class TestAdmin2 {
     clusterRegionServers.remove(i);
     // Get another server to decommission.
     serversToDecommssion.put(clusterRegionServers.get(0),
-      admin.getRegions(clusterRegionServers.get(0)));
+      ADMIN.getRegions(clusterRegionServers.get(0)));
 
     ServerName remainingServer = clusterRegionServers.get(1);
 
     // Decommission
-    admin.decommissionRegionServers(new ArrayList<ServerName>(serversToDecommssion.keySet()), true);
-    assertEquals(2, admin.listDecommissionedRegionServers().size());
+    ADMIN.decommissionRegionServers(new ArrayList<ServerName>(serversToDecommssion.keySet()), true);
+    assertEquals(2, ADMIN.listDecommissionedRegionServers().size());
 
     // Verify the regions have been off the decommissioned servers, all on the one
     // remaining server.
@@ -748,9 +734,9 @@ public class TestAdmin2 {
     for (ServerName server : serversToDecommssion.keySet()) {
       List<byte[]> encodedRegionNames = serversToDecommssion.get(server).stream()
           .map(region -> region.getEncodedNameAsBytes()).collect(Collectors.toList());
-      admin.recommissionRegionServer(server, encodedRegionNames);
+      ADMIN.recommissionRegionServer(server, encodedRegionNames);
     }
-    assertTrue(admin.listDecommissionedRegionServers().isEmpty());
+    assertTrue(ADMIN.listDecommissionedRegionServers().isEmpty());
     // Verify the regions have been moved to the recommissioned servers
     for (ServerName server : serversToDecommssion.keySet()) {
       for (RegionInfo region : serversToDecommssion.get(server)) {
@@ -770,7 +756,7 @@ public class TestAdmin2 {
       table.put(new Put(Bytes.toBytes(i)).addColumn(Bytes.toBytes("f"), Bytes.toBytes("q"),
         Bytes.toBytes(i)));
     }
-    admin.flush(tableName);
+    ADMIN.flush(tableName);
 
     HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(table.getName());
     List<HRegion> regions = rs.getRegions(tableName);
@@ -783,7 +769,7 @@ public class TestAdmin2 {
     Assert.assertNotNull(store);
     Assert.assertEquals(expectedStoreFilesSize, store.getSize());
 
-    ClusterConnection conn = ((ClusterConnection) admin.getConnection());
+    ClusterConnection conn = ((ClusterConnection) ADMIN.getConnection());
     HBaseRpcController controller = conn.getRpcControllerFactory().newController();
     for (int i = 0; i < 10; i++) {
       RegionInfo ri =
@@ -801,26 +787,26 @@ public class TestAdmin2 {
     TEST_UTIL.createTable(tableName, Bytes.toBytes("f"));
 
     // get the original table region count
-    List<RegionInfo> regions = admin.getRegions(tableName);
+    List<RegionInfo> regions = ADMIN.getRegions(tableName);
     int originalCount = regions.size();
     assertEquals(1, originalCount);
 
     // split the table and wait until region count increases
-    admin.split(tableName, Bytes.toBytes(3));
+    ADMIN.split(tableName, Bytes.toBytes(3));
     TEST_UTIL.waitFor(30000, new Predicate<Exception>() {
 
       @Override
       public boolean evaluate() throws Exception {
-        return admin.getRegions(tableName).size() > originalCount;
+        return ADMIN.getRegions(tableName).size() > originalCount;
       }
     });
 
     // do some table modification
-    TableDescriptor tableDesc = TableDescriptorBuilder.newBuilder(admin.getDescriptor(tableName))
+    TableDescriptor tableDesc = TableDescriptorBuilder.newBuilder(ADMIN.getDescriptor(tableName))
         .setMaxFileSize(11111111)
         .build();
-    admin.modifyTable(tableDesc);
-    assertEquals(11111111, admin.getDescriptor(tableName).getMaxFileSize());
+    ADMIN.modifyTable(tableDesc);
+    assertEquals(11111111, ADMIN.getDescriptor(tableName).getMaxFileSize());
   }
 
   @Test
@@ -830,7 +816,7 @@ public class TestAdmin2 {
       new byte[][] { Bytes.toBytes(3) });
 
     // assert we have at least 2 regions in the table
-    List<RegionInfo> regions = admin.getRegions(tableName);
+    List<RegionInfo> regions = ADMIN.getRegions(tableName);
     int originalCount = regions.size();
     assertTrue(originalCount >= 2);
 
@@ -838,20 +824,20 @@ public class TestAdmin2 {
     byte[] nameOfRegionB = regions.get(1).getEncodedNameAsBytes();
 
     // merge the table regions and wait until region count decreases
-    admin.mergeRegionsAsync(nameOfRegionA, nameOfRegionB, true);
+    ADMIN.mergeRegionsAsync(nameOfRegionA, nameOfRegionB, true);
     TEST_UTIL.waitFor(30000, new Predicate<Exception>() {
 
       @Override
       public boolean evaluate() throws Exception {
-        return admin.getRegions(tableName).size() < originalCount;
+        return ADMIN.getRegions(tableName).size() < originalCount;
       }
     });
 
     // do some table modification
-    TableDescriptor tableDesc = TableDescriptorBuilder.newBuilder(admin.getDescriptor(tableName))
+    TableDescriptor tableDesc = TableDescriptorBuilder.newBuilder(ADMIN.getDescriptor(tableName))
         .setMaxFileSize(11111111)
         .build();
-    admin.modifyTable(tableDesc);
-    assertEquals(11111111, admin.getDescriptor(tableName).getMaxFileSize());
+    ADMIN.modifyTable(tableDesc);
+    assertEquals(11111111, ADMIN.getDescriptor(tableName).getMaxFileSize());
   }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAlwaysSetScannerId.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAlwaysSetScannerId.java
index 2ede1dd..6c9c257 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAlwaysSetScannerId.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAlwaysSetScannerId.java
@@ -23,7 +23,6 @@ import static org.junit.Assert.assertTrue;
 import java.io.IOException;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
@@ -61,7 +60,7 @@ public class TestAlwaysSetScannerId {
 
   private static final int COUNT = 10;
 
-  private static HRegionInfo HRI;
+  private static RegionInfo HRI;
 
   private static ClientProtos.ClientService.BlockingInterface STUB;
 
@@ -73,7 +72,7 @@ public class TestAlwaysSetScannerId {
         table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i)));
       }
     }
-    HRI = UTIL.getAdmin().getTableRegions(TABLE_NAME).get(0);
+    HRI = UTIL.getAdmin().getRegions(TABLE_NAME).get(0);
     STUB = ((ConnectionImplementation) UTIL.getConnection())
         .getClient(UTIL.getHBaseCluster().getRegionServer(0).getServerName());
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java
index 0488b38..28a53cd 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java
@@ -106,7 +106,7 @@ public class TestClientTimeouts {
           admin = connection.getAdmin();
           // run some admin commands
           HBaseAdmin.available(conf);
-          admin.setBalancerRunning(false, false);
+          admin.balancerSwitch(false, false);
         } catch (MasterNotRunningException ex) {
           // Since we are randomly throwing SocketTimeoutExceptions, it is possible to get
           // a MasterNotRunningException.  It's a bug if we get other exceptions.
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestConnectionImplementation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestConnectionImplementation.java
index 9523a61..9eec04a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestConnectionImplementation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestConnectionImplementation.java
@@ -284,7 +284,7 @@ public class TestConnectionImplementation {
     TableName tableName = TableName.valueOf("HCM-testConnectionClose" + allowsInterrupt);
     TEST_UTIL.createTable(tableName, FAM_NAM).close();
 
-    boolean previousBalance = TEST_UTIL.getAdmin().setBalancerRunning(false, true);
+    boolean previousBalance = TEST_UTIL.getAdmin().balancerSwitch(false, true);
 
     Configuration c2 = new Configuration(TEST_UTIL.getConfiguration());
     // We want to work on a separate connection.
@@ -366,7 +366,7 @@ public class TestConnectionImplementation {
     table.close();
     connection.close();
     Assert.assertTrue("Unexpected exception is " + failed.get(), failed.get() == null);
-    TEST_UTIL.getAdmin().setBalancerRunning(previousBalance, true);
+    TEST_UTIL.getAdmin().balancerSwitch(previousBalance, true);
   }
 
   /**
@@ -377,7 +377,7 @@ public class TestConnectionImplementation {
     final TableName tableName = TableName.valueOf(name.getMethodName());
     TEST_UTIL.createTable(tableName, FAM_NAM).close();
     int idleTime =  20000;
-    boolean previousBalance = TEST_UTIL.getAdmin().setBalancerRunning(false, true);
+    boolean previousBalance = TEST_UTIL.getAdmin().balancerSwitch(false, true);
 
     Configuration c2 = new Configuration(TEST_UTIL.getConfiguration());
     // We want to work on a separate connection.
@@ -425,7 +425,7 @@ public class TestConnectionImplementation {
 
     connection.close();
     EnvironmentEdgeManager.reset();
-    TEST_UTIL.getAdmin().setBalancerRunning(previousBalance, true);
+    TEST_UTIL.getAdmin().balancerSwitch(previousBalance, true);
   }
 
     /**
@@ -438,7 +438,7 @@ public class TestConnectionImplementation {
     final TableName tableName = TableName.valueOf(name.getMethodName());
 
     TEST_UTIL.createTable(tableName, FAM_NAM).close();
-    boolean previousBalance = TEST_UTIL.getAdmin().setBalancerRunning(false, true);
+    boolean previousBalance = TEST_UTIL.getAdmin().balancerSwitch(false, true);
 
     Configuration c2 = new Configuration(TEST_UTIL.getConfiguration());
     // We want to work on a separate connection.
@@ -493,7 +493,7 @@ public class TestConnectionImplementation {
     } finally {
       syncBlockingFilter.set(true);
       t.join();
-      TEST_UTIL.getAdmin().setBalancerRunning(previousBalance, true);
+      TEST_UTIL.getAdmin().balancerSwitch(previousBalance, true);
     }
 
     table.close();
@@ -572,7 +572,7 @@ public class TestConnectionImplementation {
     assertNotNull(conn.getCachedLocation(TABLE_NAME, ROW));
     assertNotNull(conn.getCachedLocation(TableName.valueOf(TABLE_NAME.getName()), ROW.clone()));
 
-    TEST_UTIL.getAdmin().setBalancerRunning(false, false);
+    TEST_UTIL.getAdmin().balancerSwitch(false, false);
     HMaster master = TEST_UTIL.getMiniHBaseCluster().getMaster();
 
     // We can wait for all regions to be online, that makes log reading easier when debugging
@@ -874,7 +874,7 @@ public class TestConnectionImplementation {
       conn.clearRegionCache(TABLE_NAME3);
       Assert.assertEquals(0, conn.getNumberOfCachedRegionLocations(TABLE_NAME3));
 
-      TEST_UTIL.getAdmin().setBalancerRunning(false, false);
+      TEST_UTIL.getAdmin().balancerSwitch(false, false);
       HMaster master = TEST_UTIL.getMiniHBaseCluster().getMaster();
 
       // We can wait for all regions to be online, that makes log reading easier when debugging
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
index 0b8e2ad..5a285ef 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
@@ -29,7 +29,6 @@ import static org.junit.Assert.fail;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collections;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -4211,15 +4210,14 @@ public class TestFromClientSide {
       TEST_UTIL.createTable(tables[i], FAMILY);
     }
     Admin admin = TEST_UTIL.getAdmin();
-    HTableDescriptor[] ts = admin.listTables();
-    HashSet<HTableDescriptor> result = new HashSet<HTableDescriptor>(ts.length);
-    Collections.addAll(result, ts);
+    List<TableDescriptor> ts = admin.listTableDescriptors();
+    HashSet<TableDescriptor> result = new HashSet<>(ts);
     int size = result.size();
     assertTrue(size >= tables.length);
     for (int i = 0; i < tables.length && i < size; i++) {
       boolean found = false;
-      for (int j = 0; j < ts.length; j++) {
-        if (ts[j].getTableName().equals(tables[i])) {
+      for (int j = 0; j < ts.size(); j++) {
+        if (ts.get(j).getTableName().equals(tables[i])) {
           found = true;
           break;
         }
@@ -4334,7 +4332,7 @@ public class TestFromClientSide {
     for (HColumnDescriptor c : desc.getFamilies())
       c.setValue(attrName, attrValue);
     // update metadata for all regions of this table
-    admin.modifyTable(tableAname, desc);
+    admin.modifyTable(desc);
     // enable the table
     admin.enableTable(tableAname);
 
@@ -6364,7 +6362,7 @@ public class TestFromClientSide {
     byte[][] KEYS = HBaseTestingUtility.KEYS_FOR_HBA_CREATE_TABLE;
     Admin admin = TEST_UTIL.getAdmin();
     admin.createTable(htd, KEYS);
-    List<HRegionInfo> regions = admin.getTableRegions(htd.getTableName());
+    List<RegionInfo> regions = admin.getRegions(htd.getTableName());
 
     HRegionLocator locator =
         (HRegionLocator) admin.getConnection().getRegionLocator(htd.getTableName());
@@ -6372,7 +6370,7 @@ public class TestFromClientSide {
       List<RegionLocations> regionLocations = new ArrayList<>();
 
       // mock region locations coming from meta with multiple replicas
-      for (HRegionInfo region : regions) {
+      for (RegionInfo region : regions) {
         HRegionLocation[] arr = new HRegionLocation[regionReplication];
         for (int i = 0; i < arr.length; i++) {
           arr[i] = new HRegionLocation(RegionReplicaUtil.getRegionInfoForReplica(region, i), null);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java
index 1315d4a..2d021bb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java
@@ -64,7 +64,6 @@ import org.apache.hadoop.hbase.regionserver.RegionScanner;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.Pair;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Assert;
@@ -121,23 +120,17 @@ public class TestFromClientSide3 {
     TEST_UTIL.shutdownMiniCluster();
   }
 
-  /**
-   * @throws java.lang.Exception
-   */
   @Before
   public void setUp() throws Exception {
     // Nothing to do.
   }
 
-  /**
-   * @throws java.lang.Exception
-   */
   @After
   public void tearDown() throws Exception {
-    for (HTableDescriptor htd: TEST_UTIL.getAdmin().listTables()) {
+    for (TableDescriptor htd : TEST_UTIL.getAdmin().listTableDescriptors()) {
       LOG.info("Tear down, remove table=" + htd.getTableName());
       TEST_UTIL.deleteTable(htd.getTableName());
-  }
+    }
   }
 
   private void randomCFPuts(Table table, byte[] row, byte[] family, int nPuts)
@@ -325,12 +318,7 @@ public class TestFromClientSide3 {
       LOG.info("hbase.hstore.compaction.min should now be 5");
       HTableDescriptor htd = new HTableDescriptor(hTable.getTableDescriptor());
       htd.setValue("hbase.hstore.compaction.min", String.valueOf(5));
-      admin.modifyTable(tableName, htd);
-      Pair<Integer, Integer> st;
-      while (null != (st = admin.getAlterStatus(tableName)) && st.getFirst() > 0) {
-        LOG.debug(st.getFirst() + " regions left to update");
-        Thread.sleep(40);
-      }
+      admin.modifyTable(htd);
       LOG.info("alter status finished");
 
       // Create 3 more store files.
@@ -352,11 +340,7 @@ public class TestFromClientSide3 {
       HColumnDescriptor hcd = new HColumnDescriptor(htd.getFamily(FAMILY));
       hcd.setValue("hbase.hstore.compaction.min", String.valueOf(2));
       htd.modifyFamily(hcd);
-      admin.modifyTable(tableName, htd);
-      while (null != (st = admin.getAlterStatus(tableName)) && st.getFirst() > 0) {
-        LOG.debug(st.getFirst() + " regions left to update");
-        Thread.sleep(40);
-      }
+      admin.modifyTable(htd);
       LOG.info("alter status finished");
 
       // Issue a compaction request
@@ -387,11 +371,7 @@ public class TestFromClientSide3 {
       hcd = new HColumnDescriptor(htd.getFamily(FAMILY));
       hcd.setValue("hbase.hstore.compaction.min", null);
       htd.modifyFamily(hcd);
-      admin.modifyTable(tableName, htd);
-      while (null != (st = admin.getAlterStatus(tableName)) && st.getFirst() > 0) {
-        LOG.debug(st.getFirst() + " regions left to update");
-        Thread.sleep(40);
-      }
+      admin.modifyTable(htd);
       LOG.info("alter status finished");
       assertNull(hTable.getTableDescriptor().getFamily(FAMILY).getValue(
           "hbase.hstore.compaction.min"));
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java
index a1026a9..59ccc47 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java
@@ -129,7 +129,7 @@ public class TestHBaseAdminNoCluster {
     testMasterOperationIsRetried(new MethodCaller() {
       @Override
       public void call(Admin admin) throws Exception {
-        admin.listTables();
+        admin.listTableDescriptors();
       }
       @Override
       public void verify(MasterKeepAliveConnection masterAdmin, int count) throws Exception {
@@ -153,11 +153,11 @@ public class TestHBaseAdminNoCluster {
       }
     });
 
-    // Admin.getTableDescriptor()
+    // Admin.getDescriptor()
     testMasterOperationIsRetried(new MethodCaller() {
       @Override
       public void call(Admin admin) throws Exception {
-        admin.getTableDescriptor(TableName.valueOf(name.getMethodName()));
+        admin.getDescriptor(TableName.valueOf(name.getMethodName()));
       }
       @Override
       public void verify(MasterKeepAliveConnection masterAdmin, int count) throws Exception {
@@ -171,7 +171,7 @@ public class TestHBaseAdminNoCluster {
     testMasterOperationIsRetried(new MethodCaller() {
       @Override
       public void call(Admin admin) throws Exception {
-        admin.getTableDescriptorsByTableName(new ArrayList<>());
+        admin.listTableDescriptors(new ArrayList<>());
       }
       @Override
       public void verify(MasterKeepAliveConnection masterAdmin, int count) throws Exception {
@@ -209,11 +209,11 @@ public class TestHBaseAdminNoCluster {
       }
     });
 
-    // Admin.setBalancerRunning()
+    // Admin.balancerSwitch()
     testMasterOperationIsRetried(new MethodCaller() {
       @Override
       public void call(Admin admin) throws Exception {
-        admin.setBalancerRunning(true, true);
+        admin.balancerSwitch(true, true);
       }
       @Override
       public void verify(MasterKeepAliveConnection masterAdmin, int count) throws Exception {
@@ -223,11 +223,11 @@ public class TestHBaseAdminNoCluster {
       }
     });
 
-    // Admin.balancer()
+    // Admin.balance()
     testMasterOperationIsRetried(new MethodCaller() {
       @Override
       public void call(Admin admin) throws Exception {
-        admin.balancer();
+        admin.balance();
       }
       @Override
       public void verify(MasterKeepAliveConnection masterAdmin, int count) throws Exception {
@@ -241,7 +241,7 @@ public class TestHBaseAdminNoCluster {
     testMasterOperationIsRetried(new MethodCaller() {
       @Override
       public void call(Admin admin) throws Exception {
-        admin.enableCatalogJanitor(true);
+        admin.catalogJanitorSwitch(true);
       }
       @Override
       public void verify(MasterKeepAliveConnection masterAdmin, int count) throws Exception {
@@ -255,7 +255,7 @@ public class TestHBaseAdminNoCluster {
     testMasterOperationIsRetried(new MethodCaller() {
       @Override
       public void call(Admin admin) throws Exception {
-        admin.runCatalogScan();
+        admin.runCatalogJanitor();
       }
       @Override
       public void verify(MasterKeepAliveConnection masterAdmin, int count) throws Exception {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestLeaseRenewal.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestLeaseRenewal.java
index 3e499bd..dfe147d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestLeaseRenewal.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestLeaseRenewal.java
@@ -26,7 +26,6 @@ import org.apache.hadoop.hbase.CompatibilityFactory;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.ipc.MetricsHBaseServerSource;
 import org.apache.hadoop.hbase.test.MetricsAssertHelper;
@@ -84,20 +83,14 @@ public class TestLeaseRenewal {
     TEST_UTIL.shutdownMiniCluster();
   }
 
-  /**
-   * @throws java.lang.Exception
-   */
   @Before
   public void setUp() throws Exception {
     // Nothing to do.
   }
 
-  /**
-   * @throws java.lang.Exception
-   */
   @After
   public void tearDown() throws Exception {
-    for (HTableDescriptor htd : TEST_UTIL.getAdmin().listTables()) {
+    for (TableDescriptor htd : TEST_UTIL.getAdmin().listTableDescriptors()) {
       LOG.info("Tear down, remove table=" + htd.getTableName());
       TEST_UTIL.deleteTable(htd.getTableName());
     }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
index 6c633a2..1c2814f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
@@ -157,7 +157,7 @@ public class TestMetaWithReplicas {
 
   @Test
   public void testMetaHTDReplicaCount() throws Exception {
-    assertTrue(TEST_UTIL.getAdmin().getTableDescriptor(TableName.META_TABLE_NAME)
+    assertTrue(TEST_UTIL.getAdmin().getDescriptor(TableName.META_TABLE_NAME)
         .getRegionReplication() == 3);
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobCloneSnapshotFromClientCloneLinksAfterDelete.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobCloneSnapshotFromClientCloneLinksAfterDelete.java
index 1cae7a0..b3b4e06 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobCloneSnapshotFromClientCloneLinksAfterDelete.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobCloneSnapshotFromClientCloneLinksAfterDelete.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.util.Optional;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
@@ -112,14 +113,14 @@ public class TestMobCloneSnapshotFromClientCloneLinksAfterDelete
     delayFlush = true;
     SnapshotTestingUtils.loadData(TEST_UTIL, tableName, 20, FAMILY);
     long tid = System.currentTimeMillis();
-    byte[] snapshotName3 = Bytes.toBytes("snaptb3-" + tid);
+    String snapshotName3 = "snaptb3-" + tid;
     TableName clonedTableName3 =
       TableName.valueOf(name.getMethodName() + System.currentTimeMillis());
     admin.snapshot(snapshotName3, tableName);
     delayFlush = false;
     int snapshot3Rows = -1;
     try (Table table = TEST_UTIL.getConnection().getTable(tableName)) {
-      snapshot3Rows = TEST_UTIL.countRows(table);
+      snapshot3Rows = HBaseTestingUtility.countRows(table);
     }
     admin.cloneSnapshot(snapshotName3, clonedTableName3);
     admin.deleteSnapshot(snapshotName3);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiRespectsLimits.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiRespectsLimits.java
index a460fee..bf54449 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiRespectsLimits.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiRespectsLimits.java
@@ -96,7 +96,7 @@ public class TestMultiRespectsLimits {
       TEST_UTIL.waitFor(60000, new Waiter.Predicate<Exception>() {
         @Override
         public boolean evaluate() throws Exception {
-          return admin.getTableRegions(tableName).size() > 1;
+          return admin.getRegions(tableName).size() > 1;
         }
       });
     }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
index a42b26d..89430f6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
@@ -670,7 +670,7 @@ public class TestReplicaWithCluster {
   // within configured hbase.client.metaReplicaCallTimeout.scan from primary meta region.
   @Test
   public void testGetRegionLocationFromPrimaryMetaRegion() throws IOException, InterruptedException {
-    HTU.getAdmin().setBalancerRunning(false, true);
+    HTU.getAdmin().balancerSwitch(false, true);
 
     ((ConnectionImplementation) HTU.getAdmin().getConnection()).setUseMetaReplicas(true);
 
@@ -690,7 +690,7 @@ public class TestReplicaWithCluster {
     } finally {
       RegionServerHostingPrimayMetaRegionSlowOrStopCopro.slowDownPrimaryMetaScan = false;
       ((ConnectionImplementation) HTU.getAdmin().getConnection()).setUseMetaReplicas(false);
-      HTU.getAdmin().setBalancerRunning(true, true);
+      HTU.getAdmin().balancerSwitch(true, true);
       HTU.getAdmin().disableTable(hdt.getTableName());
       HTU.deleteTable(hdt.getTableName());
     }
@@ -703,7 +703,7 @@ public class TestReplicaWithCluster {
   // with the primary meta region.
   @Test
   public void testReplicaGetWithPrimaryAndMetaDown() throws IOException, InterruptedException {
-    HTU.getAdmin().setBalancerRunning(false, true);
+    HTU.getAdmin().balancerSwitch(false, true);
 
     ((ConnectionImplementation)HTU.getAdmin().getConnection()).setUseMetaReplicas(true);
 
@@ -789,7 +789,7 @@ public class TestReplicaWithCluster {
     } finally {
       ((ConnectionImplementation)HTU.getAdmin().getConnection()).setUseMetaReplicas(false);
       RegionServerHostingPrimayMetaRegionSlowOrStopCopro.throwException = false;
-      HTU.getAdmin().setBalancerRunning(true, true);
+      HTU.getAdmin().balancerSwitch(true, true);
       HTU.getAdmin().disableTable(hdt.getTableName());
       HTU.deleteTable(hdt.getTableName());
     }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScanWithoutFetchingData.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScanWithoutFetchingData.java
index f5c2176..b3295ac 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScanWithoutFetchingData.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScanWithoutFetchingData.java
@@ -23,7 +23,6 @@ import static org.junit.Assert.assertTrue;
 import java.io.IOException;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.ipc.HBaseRpcController;
 import org.apache.hadoop.hbase.ipc.HBaseRpcControllerImpl;
@@ -65,7 +64,7 @@ public class TestScanWithoutFetchingData {
 
   private static final int COUNT = 10;
 
-  private static HRegionInfo HRI;
+  private static RegionInfo HRI;
 
   private static ClientProtos.ClientService.BlockingInterface STUB;
 
@@ -77,7 +76,7 @@ public class TestScanWithoutFetchingData {
         table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i)));
       }
     }
-    HRI = UTIL.getAdmin().getTableRegions(TABLE_NAME).get(0);
+    HRI = UTIL.getAdmin().getRegions(TABLE_NAME).get(0);
     STUB = ((ConnectionImplementation) UTIL.getConnection())
         .getClient(UTIL.getHBaseCluster().getRegionServer(0).getServerName());
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java
index 0562c90..6c58473 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java
@@ -26,7 +26,6 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
@@ -80,7 +79,7 @@ public class TestSnapshotCloneIndependence {
   private TableName cloneTableName;
   private int countOriginalTable;
   String snapshotNameAsString;
-  byte[] snapshotName;
+  String snapshotName;
 
   /**
    * Setup the config for the cluster and start it
@@ -127,7 +126,7 @@ public class TestSnapshotCloneIndependence {
     originalTableName = TableName.valueOf("test" + testName.getMethodName());
     cloneTableName = TableName.valueOf("test-clone-" + originalTableName);
     snapshotNameAsString = "snapshot_" + originalTableName;
-    snapshotName = Bytes.toBytes(snapshotNameAsString);
+    snapshotName = snapshotNameAsString;
 
     originalTable = createTable(originalTableName, TEST_FAM);
     loadData(originalTable, TEST_FAM);
@@ -297,21 +296,21 @@ public class TestSnapshotCloneIndependence {
    */
   private void runTestRegionOperationsIndependent() throws Exception {
     // Verify that region information is the same pre-split
-    ((ClusterConnection) UTIL.getConnection()).clearRegionCache();
-    List<HRegionInfo> originalTableHRegions = admin.getTableRegions(originalTableName);
+    UTIL.getConnection().clearRegionLocationCache();
+    List<RegionInfo> originalTableHRegions = admin.getRegions(originalTableName);
 
     final int originalRegionCount = originalTableHRegions.size();
-    final int cloneTableRegionCount = admin.getTableRegions(cloneTableName).size();
+    final int cloneTableRegionCount = admin.getRegions(cloneTableName).size();
     Assert.assertEquals(
       "The number of regions in the cloned table is different than in the original table.",
       originalRegionCount, cloneTableRegionCount);
 
     // Split a region on the parent table
-    admin.splitRegion(originalTableHRegions.get(0).getRegionName());
+    admin.splitRegionAsync(originalTableHRegions.get(0).getRegionName(), null).get();
     waitOnSplit(UTIL.getConnection(), originalTable, originalRegionCount);
 
     // Verify that the cloned table region is not split
-    final int cloneTableRegionCount2 = admin.getTableRegions(cloneTableName).size();
+    final int cloneTableRegionCount2 = admin.getRegions(cloneTableName).size();
     Assert.assertEquals(
       "The number of regions in the cloned table changed though none of its regions were split.",
       cloneTableRegionCount, cloneTableRegionCount2);
@@ -335,8 +334,9 @@ public class TestSnapshotCloneIndependence {
     // get a description of the cloned table
     // get a list of its families
     // assert that the family is there
-    HTableDescriptor originalTableDescriptor = originalTable.getTableDescriptor();
-    HTableDescriptor clonedTableDescriptor = admin.getTableDescriptor(cloneTableName);
+    HTableDescriptor originalTableDescriptor = new HTableDescriptor(originalTable.getDescriptor());
+    HTableDescriptor clonedTableDescriptor =
+      new HTableDescriptor(admin.getDescriptor(cloneTableName));
 
     Assert.assertTrue("The original family was not found. There is something wrong. ",
       originalTableDescriptor.hasFamily(TEST_FAM));
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java
index b46404f..ad4b84c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java
@@ -146,19 +146,10 @@ public class TestSnapshotFromClient {
 
   /**
    * Test snapshotting not allowed hbase:meta and -ROOT-
-   * @throws Exception
    */
-  @Test
+  @Test(expected = IllegalArgumentException.class)
   public void testMetaTablesSnapshot() throws Exception {
-    Admin admin = UTIL.getAdmin();
-    byte[] snapshotName = Bytes.toBytes("metaSnapshot");
-
-    try {
-      admin.snapshot(snapshotName, TableName.META_TABLE_NAME);
-      fail("taking a snapshot of hbase:meta should not be allowed");
-    } catch (IllegalArgumentException e) {
-      // expected
-    }
+    UTIL.getAdmin().snapshot("metaSnapshot", TableName.META_TABLE_NAME);
   }
 
   /**
@@ -177,16 +168,16 @@ public class TestSnapshotFromClient {
     UTIL.loadTable(table, TEST_FAM);
     table.close();
 
-    byte[] snapshot1 = Bytes.toBytes("TableSnapshot1");
+    String snapshot1 = "TableSnapshot1";
     admin.snapshot(snapshot1, TABLE_NAME);
     LOG.debug("Snapshot1 completed.");
 
-    byte[] snapshot2 = Bytes.toBytes("TableSnapshot2");
+    String snapshot2 = "TableSnapshot2";
     admin.snapshot(snapshot2, TABLE_NAME);
     LOG.debug("Snapshot2 completed.");
 
     String snapshot3 = "3rdTableSnapshot";
-    admin.snapshot(Bytes.toBytes(snapshot3), TABLE_NAME);
+    admin.snapshot(snapshot3, TABLE_NAME);
     LOG.debug(snapshot3 + " completed.");
 
     // delete the first two snapshots
@@ -260,13 +251,13 @@ public class TestSnapshotFromClient {
     // make sure the table doesn't exist
     boolean fail = false;
     do {
-    try {
-      admin.getTableDescriptor(TableName.valueOf(tableName));
-      fail = true;
-          LOG.error("Table:" + tableName + " already exists, checking a new name");
-      tableName = tableName+"!";
-    } catch (TableNotFoundException e) {
-      fail = false;
+      try {
+        admin.getDescriptor(TableName.valueOf(tableName));
+        fail = true;
+        LOG.error("Table:" + tableName + " already exists, checking a new name");
+        tableName = tableName + "!";
+      } catch (TableNotFoundException e) {
+        fail = false;
       }
     } while (fail);
 
@@ -297,7 +288,7 @@ public class TestSnapshotFromClient {
       FSUtils.getRootDir(UTIL.getConfiguration()), LOG);
 
     // take a snapshot of the disabled table
-    byte[] snapshot = Bytes.toBytes("testOfflineTableSnapshotWithEmptyRegions");
+    String snapshot = "testOfflineTableSnapshotWithEmptyRegions";
     admin.snapshot(snapshot, TABLE_NAME);
     LOG.debug("Snapshot completed.");
 
@@ -342,7 +333,7 @@ public class TestSnapshotFromClient {
       LOG.debug("Snapshot2 completed.");
 
       String table2Snapshot1 = "Table2Snapshot1";
-      admin.snapshot(Bytes.toBytes(table2Snapshot1), tableName);
+      admin.snapshot(table2Snapshot1, tableName);
       LOG.debug(table2Snapshot1 + " completed.");
 
       List<SnapshotDescription> listTableSnapshots =
@@ -384,7 +375,7 @@ public class TestSnapshotFromClient {
       LOG.debug("Snapshot2 completed.");
 
       String table2Snapshot1 = "Table2Snapshot1";
-      admin.snapshot(Bytes.toBytes(table2Snapshot1), TABLE_NAME);
+      admin.snapshot(table2Snapshot1, TABLE_NAME);
       LOG.debug(table2Snapshot1 + " completed.");
 
       List<SnapshotDescription> listTableSnapshots =
@@ -427,7 +418,7 @@ public class TestSnapshotFromClient {
       LOG.debug("Snapshot2 completed.");
 
       String table2Snapshot1 = "Table2Snapshot1";
-      admin.snapshot(Bytes.toBytes(table2Snapshot1), tableName);
+      admin.snapshot(table2Snapshot1, tableName);
       LOG.debug(table2Snapshot1 + " completed.");
 
       Pattern tableNamePattern = Pattern.compile("test.*");
@@ -459,7 +450,7 @@ public class TestSnapshotFromClient {
       LOG.debug("Snapshot2 completed.");
 
       String table2Snapshot1 = "Table2Snapshot1";
-      admin.snapshot(Bytes.toBytes(table2Snapshot1), TABLE_NAME);
+      admin.snapshot(table2Snapshot1, TABLE_NAME);
       LOG.debug(table2Snapshot1 + " completed.");
 
       admin.deleteTableSnapshots(tableNamePattern, Pattern.compile("Table1.*"));
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotMetadata.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotMetadata.java
index ef52057..1af70cb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotMetadata.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotMetadata.java
@@ -178,7 +178,7 @@ public class TestSnapshotMetadata {
     admin.createTable(htd);
     Table original = UTIL.getConnection().getTable(originalTableName);
     originalTableName = TableName.valueOf(sourceTableNameAsString);
-    originalTableDescriptor = admin.getTableDescriptor(originalTableName);
+    originalTableDescriptor = new HTableDescriptor(admin.getDescriptor(originalTableName));
     originalTableDescription = originalTableDescriptor.toStringCustomizedValues();
 
     original.close();
@@ -195,7 +195,7 @@ public class TestSnapshotMetadata {
     final TableName clonedTableName = TableName.valueOf(clonedTableNameAsString);
     final String snapshotNameAsString = "snapshot" + originalTableName
         + System.currentTimeMillis();
-    final byte[] snapshotName = Bytes.toBytes(snapshotNameAsString);
+    final String snapshotName = snapshotNameAsString;
 
     // restore the snapshot into a cloned table and examine the output
     List<byte[]> familiesList = new ArrayList<>();
@@ -207,7 +207,7 @@ public class TestSnapshotMetadata {
 
     admin.cloneSnapshot(snapshotName, clonedTableName);
     Table clonedTable = UTIL.getConnection().getTable(clonedTableName);
-    HTableDescriptor cloneHtd = admin.getTableDescriptor(clonedTableName);
+    HTableDescriptor cloneHtd = new HTableDescriptor(admin.getDescriptor(clonedTableName));
     assertEquals(
       originalTableDescription.replace(originalTableName.getNameAsString(),clonedTableNameAsString),
       cloneHtd.toStringCustomizedValues());
@@ -287,7 +287,6 @@ public class TestSnapshotMetadata {
     // take a "disabled" snapshot
     final String snapshotNameAsString = "snapshot" + originalTableName
         + System.currentTimeMillis();
-    final byte[] snapshotName = Bytes.toBytes(snapshotNameAsString);
 
     SnapshotTestingUtils.createSnapshotAndValidate(admin, originalTableName,
       familiesWithDataList, emptyFamiliesList, snapshotNameAsString, rootDir, fs,
@@ -303,7 +302,7 @@ public class TestSnapshotMetadata {
       HColumnDescriptor hcd = new HColumnDescriptor(newFamilyName);
       admin.addColumnFamily(originalTableName, hcd);
       assertTrue("New column family was not added.",
-        admin.getTableDescriptor(originalTableName).toString().contains(newFamilyNameAsString));
+        admin.getDescriptor(originalTableName).toString().contains(newFamilyNameAsString));
     }
 
     // restore it
@@ -311,16 +310,14 @@ public class TestSnapshotMetadata {
       admin.disableTable(originalTableName);
     }
 
-    admin.restoreSnapshot(snapshotName);
+    admin.restoreSnapshot(snapshotNameAsString);
     admin.enableTable(originalTableName);
 
     // verify that the descrption is reverted
-    Table original = UTIL.getConnection().getTable(originalTableName);
-    try {
-      assertTrue(originalTableDescriptor.equals(admin.getTableDescriptor(originalTableName)));
-      assertTrue(originalTableDescriptor.equals(original.getTableDescriptor()));
-    } finally {
-      original.close();
+    try (Table original = UTIL.getConnection().getTable(originalTableName)) {
+      assertEquals(originalTableDescriptor,
+        new HTableDescriptor(admin.getDescriptor(originalTableName)));
+      assertEquals(originalTableDescriptor, new HTableDescriptor(original.getDescriptor()));
     }
   }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotTemporaryDirectory.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotTemporaryDirectory.java
index f58b947..589b31d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotTemporaryDirectory.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotTemporaryDirectory.java
@@ -29,6 +29,7 @@ import java.util.Arrays;
 import java.util.Iterator;
 import java.util.List;
 import java.util.UUID;
+import java.util.regex.Pattern;
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -152,9 +153,9 @@ public class TestSnapshotTemporaryDirectory {
       throws IOException, InterruptedException {
     long tid = System.currentTimeMillis();
     TableName tableName = TableName.valueOf("testtb-" + tid);
-    byte[] emptySnapshot = Bytes.toBytes("emptySnaptb-" + tid);
-    byte[] snapshotName0 = Bytes.toBytes("snaptb0-" + tid);
-    byte[] snapshotName1 = Bytes.toBytes("snaptb1-" + tid);
+    String emptySnapshot = "emptySnaptb-" + tid;
+    String snapshotName0 = "snaptb0-" + tid;
+    String snapshotName1 = "snaptb1-" + tid;
     int snapshot0Rows;
     int snapshot1Rows;
 
@@ -163,7 +164,7 @@ public class TestSnapshotTemporaryDirectory {
     admin.disableTable(tableName);
 
     // take an empty snapshot
-    takeSnapshot(tableName, Bytes.toString(emptySnapshot), true);
+    takeSnapshot(tableName, emptySnapshot, true);
 
     // enable table and insert data
     admin.enableTable(tableName);
@@ -174,7 +175,7 @@ public class TestSnapshotTemporaryDirectory {
     admin.disableTable(tableName);
 
     // take a snapshot
-    takeSnapshot(tableName, Bytes.toString(snapshotName0), true);
+    takeSnapshot(tableName, snapshotName0, true);
 
     // enable table and insert more data
     admin.enableTable(tableName);
@@ -185,7 +186,7 @@ public class TestSnapshotTemporaryDirectory {
 
     SnapshotTestingUtils.verifyRowCount(UTIL, tableName, snapshot1Rows);
     admin.disableTable(tableName);
-    takeSnapshot(tableName, Bytes.toString(snapshotName1), true);
+    takeSnapshot(tableName, snapshotName1, true);
 
     // Restore from snapshot-0
     admin.restoreSnapshot(snapshotName0);
@@ -218,9 +219,9 @@ public class TestSnapshotTemporaryDirectory {
       throws IOException, InterruptedException {
     long tid = System.currentTimeMillis();
     TableName tableName = TableName.valueOf("testtb-" + tid);
-    byte[] emptySnapshot = Bytes.toBytes("emptySnaptb-" + tid);
-    byte[] snapshotName0 = Bytes.toBytes("snaptb0-" + tid);
-    byte[] snapshotName1 = Bytes.toBytes("snaptb1-" + tid);
+    String emptySnapshot = "emptySnaptb-" + tid;
+    String snapshotName0 = "snaptb0-" + tid;
+    String snapshotName1 = "snaptb1-" + tid;
     int snapshot0Rows;
     int snapshot1Rows;
 
@@ -228,7 +229,7 @@ public class TestSnapshotTemporaryDirectory {
     SnapshotTestingUtils.createTable(UTIL, tableName, getNumReplicas(), TEST_FAM);
 
     // take an empty snapshot
-    takeSnapshot(tableName, Bytes.toString(emptySnapshot), false);
+    takeSnapshot(tableName, emptySnapshot, false);
 
     // Insert data
     SnapshotTestingUtils.loadData(UTIL, tableName, 500, TEST_FAM);
@@ -237,7 +238,7 @@ public class TestSnapshotTemporaryDirectory {
     }
 
     // take a snapshot
-    takeSnapshot(tableName, Bytes.toString(snapshotName0), false);
+    takeSnapshot(tableName, snapshotName0, false);
 
     // Insert more data
     SnapshotTestingUtils.loadData(UTIL, tableName, 500, TEST_FAM);
@@ -246,7 +247,7 @@ public class TestSnapshotTemporaryDirectory {
     }
 
     SnapshotTestingUtils.verifyRowCount(UTIL, tableName, snapshot1Rows);
-    takeSnapshot(tableName, Bytes.toString(snapshotName1), false);
+    takeSnapshot(tableName, snapshotName1, false);
 
     // Restore from snapshot-0
     admin.disableTable(tableName);
@@ -416,12 +417,11 @@ public class TestSnapshotTemporaryDirectory {
   }
 
   // Ensures that the snapshot is transferred to the proper completed snapshot directory
-  @Test(timeout = 180000) public void testEnsureTemporaryDirectoryTransfer() throws Exception {
-    Admin admin = null;
+  @Test(timeout = 180000)
+  public void testEnsureTemporaryDirectoryTransfer() throws Exception {
+    Admin admin = UTIL.getAdmin();
     TableName tableName2 = TableName.valueOf("testListTableSnapshots");
     try {
-      admin = UTIL.getHBaseAdmin();
-
       HTableDescriptor htd = new HTableDescriptor(tableName2);
       UTIL.createTable(htd, new byte[][] { TEST_FAM }, UTIL.getConfiguration());
 
@@ -437,7 +437,8 @@ public class TestSnapshotTemporaryDirectory {
       takeSnapshot(TABLE_NAME, table2Snapshot1, false);
       LOG.debug("Table2Snapshot1 completed.");
 
-      List<SnapshotDescription> listTableSnapshots = admin.listTableSnapshots("test.*", ".*");
+      List<SnapshotDescription> listTableSnapshots =
+        admin.listTableSnapshots(Pattern.compile("test.*"), Pattern.compile(".*"));
       List<String> listTableSnapshotNames = new ArrayList<String>();
       assertEquals(3, listTableSnapshots.size());
       for (SnapshotDescription s : listTableSnapshots) {
@@ -447,16 +448,14 @@ public class TestSnapshotTemporaryDirectory {
       assertTrue(listTableSnapshotNames.contains(table1Snapshot2));
       assertTrue(listTableSnapshotNames.contains(table2Snapshot1));
     } finally {
-      if (admin != null) {
-        try {
-          admin.deleteSnapshots("Table.*");
-        } catch (SnapshotDoesNotExistException ignore) {
-        }
-        if (admin.tableExists(tableName2)) {
-          UTIL.deleteTable(tableName2);
-        }
-        admin.close();
+      try {
+        admin.deleteSnapshots(Pattern.compile("Table.*"));
+      } catch (SnapshotDoesNotExistException ignore) {
+      }
+      if (admin.tableExists(tableName2)) {
+        UTIL.deleteTable(tableName2);
       }
+      admin.close();
     }
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSplitOrMergeStatus.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSplitOrMergeStatus.java
index 64392eb..3e40b6f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSplitOrMergeStatus.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSplitOrMergeStatus.java
@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.hbase.client;
 
-import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -29,7 +28,6 @@ import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -84,20 +82,18 @@ public class TestSplitOrMergeStatus {
 
     Admin admin = TEST_UTIL.getAdmin();
     initSwitchStatus(admin);
-    boolean[] results = admin.setSplitOrMergeEnabled(false, false, MasterSwitchType.SPLIT);
-    assertEquals(1, results.length);
-    assertTrue(results[0]);
+    boolean result = admin.splitSwitch(false, false);
+    assertTrue(result);
     admin.split(t.getName());
-    int count = admin.getTableRegions(tableName).size();
+    int count = admin.getRegions(tableName).size();
     assertTrue(originalCount == count);
-    results = admin.setSplitOrMergeEnabled(true, false, MasterSwitchType.SPLIT);
-    assertEquals(1, results.length);
-    assertFalse(results[0]);
+    result = admin.splitSwitch(true, false);
+    assertFalse(result);
     admin.split(t.getName());
-    while ((count = admin.getTableRegions(tableName).size()) == originalCount) {
+    while ((count = admin.getRegions(tableName).size()) == originalCount) {
       Threads.sleep(1);
     }
-    count = admin.getTableRegions(tableName).size();
+    count = admin.getRegions(tableName).size();
     assertTrue(originalCount < count);
     admin.close();
   }
@@ -110,21 +106,20 @@ public class TestSplitOrMergeStatus {
     TEST_UTIL.loadTable(t, FAMILY, false);
 
     Admin admin = TEST_UTIL.getAdmin();
-    int originalCount = admin.getTableRegions(tableName).size();
+    int originalCount = admin.getRegions(tableName).size();
     initSwitchStatus(admin);
     admin.split(t.getName());
     int postSplitCount = -1;
-    while ((postSplitCount = admin.getTableRegions(tableName).size()) == originalCount) {
+    while ((postSplitCount = admin.getRegions(tableName).size()) == originalCount) {
       Threads.sleep(1);
     }
     assertTrue("originalCount=" + originalCount + ", newCount=" + postSplitCount,
         originalCount != postSplitCount);
 
     // Merge switch is off so merge should NOT succeed.
-    boolean[] results = admin.setSplitOrMergeEnabled(false, false, MasterSwitchType.MERGE);
-    assertEquals(1, results.length);
-    assertTrue(results[0]);
-    List<HRegionInfo> regions = admin.getTableRegions(t.getName());
+    boolean result = admin.mergeSwitch(false, false);
+    assertTrue(result);
+    List<RegionInfo> regions = admin.getRegions(t.getName());
     assertTrue(regions.size() > 1);
     Future<?> f = admin.mergeRegionsAsync(regions.get(0).getEncodedNameAsBytes(),
       regions.get(1).getEncodedNameAsBytes(), true);
@@ -134,17 +129,16 @@ public class TestSplitOrMergeStatus {
     } catch (ExecutionException ee) {
       // Expected.
     }
-    int count = admin.getTableRegions(tableName).size();
+    int count = admin.getRegions(tableName).size();
     assertTrue("newCount=" + postSplitCount + ", count=" + count, postSplitCount == count);
 
-    results = admin.setSplitOrMergeEnabled(true, false, MasterSwitchType.MERGE);
-    regions = admin.getTableRegions(t.getName());
-    assertEquals(1, results.length);
-    assertFalse(results[0]);
+    result = admin.mergeSwitch(true, false);
+    regions = admin.getRegions(t.getName());
+    assertFalse(result);
     f = admin.mergeRegionsAsync(regions.get(0).getEncodedNameAsBytes(),
       regions.get(1).getEncodedNameAsBytes(), true);
     f.get(10, TimeUnit.SECONDS);
-    count = admin.getTableRegions(tableName).size();
+    count = admin.getRegions(tableName).size();
     assertTrue((postSplitCount / 2 /*Merge*/) == count);
     admin.close();
   }
@@ -152,24 +146,22 @@ public class TestSplitOrMergeStatus {
   @Test
   public void testMultiSwitches() throws IOException {
     Admin admin = TEST_UTIL.getAdmin();
-    boolean[] switches = admin.setSplitOrMergeEnabled(false, false,
-      MasterSwitchType.SPLIT, MasterSwitchType.MERGE);
-    for (boolean s : switches){
-      assertTrue(s);
-    }
-    assertFalse(admin.isSplitOrMergeEnabled(MasterSwitchType.SPLIT));
-    assertFalse(admin.isSplitOrMergeEnabled(MasterSwitchType.MERGE));
+    assertTrue(admin.splitSwitch(false, false));
+    assertTrue(admin.mergeSwitch(false, false));
+
+    assertFalse(admin.isSplitEnabled());
+    assertFalse(admin.isMergeEnabled());
     admin.close();
   }
 
   private void initSwitchStatus(Admin admin) throws IOException {
-    if (!admin.isSplitOrMergeEnabled(MasterSwitchType.SPLIT)) {
-      admin.setSplitOrMergeEnabled(true, false, MasterSwitchType.SPLIT);
+    if (!admin.isSplitEnabled()) {
+      admin.splitSwitch(true, false);
     }
-    if (!admin.isSplitOrMergeEnabled(MasterSwitchType.MERGE)) {
-      admin.setSplitOrMergeEnabled(true, false, MasterSwitchType.MERGE);
+    if (!admin.isMergeEnabled()) {
+      admin.mergeSwitch(true, false);
     }
-    assertTrue(admin.isSplitOrMergeEnabled(MasterSwitchType.SPLIT));
-    assertTrue(admin.isSplitOrMergeEnabled(MasterSwitchType.MERGE));
+    assertTrue(admin.isSplitEnabled());
+    assertTrue(admin.isMergeEnabled());
   }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTableFavoredNodes.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTableFavoredNodes.java
index 716b1dc..fd4cc51 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTableFavoredNodes.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTableFavoredNodes.java
@@ -114,8 +114,8 @@ public class TestTableFavoredNodes {
   public void setup() throws IOException {
     fnm = TEST_UTIL.getMiniHBaseCluster().getMaster().getFavoredNodesManager();
     admin = TEST_UTIL.getAdmin();
-    admin.setBalancerRunning(false, true);
-    admin.enableCatalogJanitor(false);
+    admin.balancerSwitch(false, true);
+    admin.catalogJanitorSwitch(false);
   }
 
   /*
@@ -130,7 +130,7 @@ public class TestTableFavoredNodes {
     // All regions should have favored nodes
     checkIfFavoredNodeInformationIsCorrect(tableName);
 
-    List<HRegionInfo> regions = admin.getTableRegions(tableName);
+    List<RegionInfo> regions = admin.getRegions(tableName);
 
     TEST_UTIL.deleteTable(tableName);
 
@@ -149,13 +149,13 @@ public class TestTableFavoredNodes {
     // All regions should have favored nodes
     checkIfFavoredNodeInformationIsCorrect(tableName);
 
-    List<HRegionInfo> regions = admin.getTableRegions(tableName);
+    List<RegionInfo> regions = admin.getRegions(tableName);
     TEST_UTIL.truncateTable(tableName, true);
 
     checkNoFNForDeletedTable(regions);
     checkIfFavoredNodeInformationIsCorrect(tableName);
 
-    regions = admin.getTableRegions(tableName);
+    regions = admin.getRegions(tableName);
     TEST_UTIL.truncateTable(tableName, false);
     checkNoFNForDeletedTable(regions);
 
@@ -170,7 +170,7 @@ public class TestTableFavoredNodes {
     final TableName tableName = TableName.valueOf(name.getMethodName());
     Table t = TEST_UTIL.createTable(tableName, Bytes.toBytes("f"), splitKeys);
     TEST_UTIL.waitUntilAllRegionsAssigned(tableName);
-    final int numberOfRegions = admin.getTableRegions(t.getName()).size();
+    final int numberOfRegions = admin.getRegions(t.getName()).size();
 
     checkIfFavoredNodeInformationIsCorrect(tableName);
 
@@ -211,14 +211,14 @@ public class TestTableFavoredNodes {
 
     // Major compact table and run catalog janitor. Parent's FN should be removed
     TEST_UTIL.getMiniHBaseCluster().compact(tableName, true);
-    admin.runCatalogScan();
+    admin.runCatalogJanitor();
     // Catalog cleanup is async. Wait on procedure to finish up.
     ProcedureTestingUtility.waitAllProcedures(
         TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor());
     // assertEquals("Parent region should have been cleaned", 1, admin.runCatalogScan());
     assertNull("Parent FN should be null", fnm.getFavoredNodes(parent));
 
-    List<HRegionInfo> regions = admin.getTableRegions(tableName);
+    List<RegionInfo> regions = admin.getRegions(tableName);
     // Split and Table Disable interfere with each other around region replicas
     // TODO. Meantime pause a few seconds.
     Threads.sleep(2000);
@@ -266,22 +266,22 @@ public class TestTableFavoredNodes {
 
     // Major compact table and run catalog janitor. Parent FN should be removed
     TEST_UTIL.getMiniHBaseCluster().compact(tableName, true);
-    assertEquals("Merge parents should have been cleaned", 1, admin.runCatalogScan());
+    assertEquals("Merge parents should have been cleaned", 1, admin.runCatalogJanitor());
     // Catalog cleanup is async. Wait on procedure to finish up.
     ProcedureTestingUtility.waitAllProcedures(
         TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor());
     assertNull("Parent FN should be null", fnm.getFavoredNodes(regionA));
     assertNull("Parent FN should be null", fnm.getFavoredNodes(regionB));
 
-    List<HRegionInfo> regions = admin.getTableRegions(tableName);
+    List<RegionInfo> regions = admin.getRegions(tableName);
 
     TEST_UTIL.deleteTable(tableName);
 
     checkNoFNForDeletedTable(regions);
   }
 
-  private void checkNoFNForDeletedTable(List<HRegionInfo> regions) {
-    for (HRegionInfo region : regions) {
+  private void checkNoFNForDeletedTable(List<RegionInfo> regions) {
+    for (RegionInfo region : regions) {
       LOG.info("Testing if FN data for " + region);
       assertNull("FN not null for deleted table's region: " + region, fnm.getFavoredNodes(region));
     }
@@ -362,8 +362,8 @@ public class TestTableFavoredNodes {
 
     for (TableName sysTable :
         admin.listTableNamesByNamespace(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR)) {
-      List<HRegionInfo> regions = admin.getTableRegions(sysTable);
-      for (HRegionInfo region : regions) {
+      List<RegionInfo> regions = admin.getRegions(sysTable);
+      for (RegionInfo region : regions) {
         assertNull("FN should be null for sys region", fnm.getFavoredNodes(region));
       }
     }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTableSnapshotScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTableSnapshotScanner.java
index e999f8c..720ce2d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTableSnapshotScanner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTableSnapshotScanner.java
@@ -27,7 +27,6 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.StartMiniClusterOption;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
@@ -65,7 +64,7 @@ public class TestTableSnapshotScanner {
   public static void blockUntilSplitFinished(HBaseTestingUtility util, TableName tableName,
       int expectedRegionSize) throws Exception {
     for (int i = 0; i < 100; i++) {
-      List<HRegionInfo> hRegionInfoList = util.getAdmin().getTableRegions(tableName);
+      List<RegionInfo> hRegionInfoList = util.getAdmin().getRegions(tableName);
       if (hRegionInfoList.size() >= expectedRegionSize) {
         break;
       }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminWithClusters.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminWithClusters.java
index 0ff757e..46536c1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminWithClusters.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminWithClusters.java
@@ -93,15 +93,15 @@ public class TestReplicationAdminWithClusters extends TestReplicationBase {
 
   @Test
   public void disableNotFullReplication() throws Exception {
-    HTableDescriptor table = new HTableDescriptor(admin2.getTableDescriptor(tableName));
+    HTableDescriptor table = new HTableDescriptor(admin2.getDescriptor(tableName));
     HColumnDescriptor f = new HColumnDescriptor("notReplicatedFamily");
     table.addFamily(f);
     admin1.disableTable(tableName);
-    admin1.modifyTable(tableName, table);
+    admin1.modifyTable(table);
     admin1.enableTable(tableName);
 
     admin1.disableTableReplication(tableName);
-    table = admin1.getTableDescriptor(tableName);
+    table = new HTableDescriptor(admin1.getDescriptor(tableName));
     for (HColumnDescriptor fam : table.getColumnFamilies()) {
       assertEquals(HConstants.REPLICATION_SCOPE_LOCAL, fam.getScope());
     }
@@ -121,20 +121,20 @@ public class TestReplicationAdminWithClusters extends TestReplicationBase {
 
   @Test
   public void testEnableReplicationWhenReplicationNotEnabled() throws Exception {
-    HTableDescriptor table = new HTableDescriptor(admin1.getTableDescriptor(tableName));
+    HTableDescriptor table = new HTableDescriptor(admin1.getDescriptor(tableName));
     for (HColumnDescriptor fam : table.getColumnFamilies()) {
       fam.setScope(HConstants.REPLICATION_SCOPE_LOCAL);
     }
     admin1.disableTable(tableName);
-    admin1.modifyTable(tableName, table);
+    admin1.modifyTable(table);
     admin1.enableTable(tableName);
 
     admin2.disableTable(tableName);
-    admin2.modifyTable(tableName, table);
+    admin2.modifyTable(table);
     admin2.enableTable(tableName);
 
     admin1.enableTableReplication(tableName);
-    table = admin1.getTableDescriptor(tableName);
+    table = new HTableDescriptor(admin1.getDescriptor(tableName));
     for (HColumnDescriptor fam : table.getColumnFamilies()) {
       assertEquals(HConstants.REPLICATION_SCOPE_GLOBAL, fam.getScope());
     }
@@ -142,11 +142,11 @@ public class TestReplicationAdminWithClusters extends TestReplicationBase {
 
   @Test
   public void testEnableReplicationWhenTableDescriptorIsNotSameInClusters() throws Exception {
-    HTableDescriptor table = new HTableDescriptor(admin2.getTableDescriptor(tableName));
+    HTableDescriptor table = new HTableDescriptor(admin2.getDescriptor(tableName));
     HColumnDescriptor f = new HColumnDescriptor("newFamily");
     table.addFamily(f);
     admin2.disableTable(tableName);
-    admin2.modifyTable(tableName, table);
+    admin2.modifyTable(table);
     admin2.enableTable(tableName);
 
     try {
@@ -156,10 +156,10 @@ public class TestReplicationAdminWithClusters extends TestReplicationBase {
 
     }
     admin1.disableTable(tableName);
-    admin1.modifyTable(tableName, table);
+    admin1.modifyTable(table);
     admin1.enableTable(tableName);
     admin1.enableTableReplication(tableName);
-    table = admin1.getTableDescriptor(tableName);
+    table = new HTableDescriptor(admin1.getDescriptor(tableName));
     for (HColumnDescriptor fam : table.getColumnFamilies()) {
       assertEquals(HConstants.REPLICATION_SCOPE_GLOBAL, fam.getScope());
     }
@@ -171,12 +171,12 @@ public class TestReplicationAdminWithClusters extends TestReplicationBase {
   @Test
   public void testDisableAndEnableReplication() throws Exception {
     admin1.disableTableReplication(tableName);
-    HTableDescriptor table = admin1.getTableDescriptor(tableName);
+    HTableDescriptor table = new HTableDescriptor(admin1.getDescriptor(tableName));
     for (HColumnDescriptor fam : table.getColumnFamilies()) {
       assertEquals(HConstants.REPLICATION_SCOPE_LOCAL, fam.getScope());
     }
     admin1.enableTableReplication(tableName);
-    table = admin1.getTableDescriptor(tableName);
+    table = new HTableDescriptor(admin1.getDescriptor(tableName));
     for (HColumnDescriptor fam : table.getColumnFamilies()) {
       assertEquals(HConstants.REPLICATION_SCOPE_GLOBAL, fam.getScope());
     }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorMetrics.java
index 24f74d8..473da0b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorMetrics.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorMetrics.java
@@ -286,7 +286,7 @@ public class TestCoprocessorMetrics {
   public void setup() throws IOException {
     try (Connection connection = ConnectionFactory.createConnection(UTIL.getConfiguration());
          Admin admin = connection.getAdmin()) {
-      for (HTableDescriptor htd : admin.listTables()) {
+      for (TableDescriptor htd : admin.listTableDescriptors()) {
         UTIL.deleteTable(htd.getTableName());
       }
     }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
index 58f4b9b..178429d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
@@ -1256,11 +1256,11 @@ public class TestMasterObserver {
   }
 
   private static HBaseTestingUtility UTIL = new HBaseTestingUtility();
-  private static byte[] TEST_SNAPSHOT = Bytes.toBytes("observed_snapshot");
+  private static String TEST_SNAPSHOT = "observed_snapshot";
   private static TableName TEST_CLONE = TableName.valueOf("observed_clone");
   private static byte[] TEST_FAMILY = Bytes.toBytes("fam1");
-  private static byte[] TEST_FAMILY2 = Bytes.toBytes("fam2");
-  @Rule public TestName name = new TestName();
+  @Rule
+  public TestName name = new TestName();
 
   @BeforeClass
   public static void setupBeforeClass() throws Exception {
@@ -1507,10 +1507,10 @@ public class TestMasterObserver {
 
   private void modifyTableSync(Admin admin, TableName tableName, HTableDescriptor htd)
       throws IOException {
-    admin.modifyTable(tableName, htd);
+    admin.modifyTable(htd);
     //wait until modify table finishes
     for (int t = 0; t < 100; t++) { //10 sec timeout
-      HTableDescriptor td = admin.getTableDescriptor(htd.getTableName());
+      HTableDescriptor td = new HTableDescriptor(admin.getDescriptor(htd.getTableName()));
       if (td.equals(htd)) {
         break;
       }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestLoadAndSwitchEncodeOnDisk.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestLoadAndSwitchEncodeOnDisk.java
index b607ca7..ac46d9b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestLoadAndSwitchEncodeOnDisk.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestLoadAndSwitchEncodeOnDisk.java
@@ -22,9 +22,9 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.List;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
-import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
@@ -80,7 +80,7 @@ public class TestLoadAndSwitchEncodeOnDisk extends
     compression = Compression.Algorithm.GZ; // used for table setup
     super.loadTest();
 
-    HColumnDescriptor hcd = getColumnDesc(admin);
+    ColumnFamilyDescriptor hcd = getColumnDesc(admin);
     System.err.println("\nDisabling encode-on-disk. Old column descriptor: " + hcd + "\n");
     Table t = TEST_UTIL.getConnection().getTable(TABLE);
     assertAllOnLine(t);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitorInMemoryStates.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitorInMemoryStates.java
index fcd1638..78911c0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitorInMemoryStates.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitorInMemoryStates.java
@@ -24,6 +24,7 @@ import static org.junit.Assert.assertTrue;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.concurrent.ExecutionException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
@@ -91,13 +92,14 @@ public class TestCatalogJanitorInMemoryStates {
    * Test clearing a split parent from memory.
    */
   @Test
-  public void testInMemoryParentCleanup() throws IOException, InterruptedException {
+  public void testInMemoryParentCleanup()
+      throws IOException, InterruptedException, ExecutionException {
     final AssignmentManager am = TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager();
     final ServerManager sm = TEST_UTIL.getHBaseCluster().getMaster().getServerManager();
     final CatalogJanitor janitor = TEST_UTIL.getHBaseCluster().getMaster().getCatalogJanitor();
 
     Admin admin = TEST_UTIL.getAdmin();
-    admin.enableCatalogJanitor(false);
+    admin.catalogJanitorSwitch(false);
 
     final TableName tableName = TableName.valueOf(name.getMethodName());
     Table t = TEST_UTIL.createTable(tableName, FAMILY);
@@ -129,19 +131,18 @@ public class TestCatalogJanitorInMemoryStates {
 
   }
 
-  /*
- * Splits a region
- * @param t Region to split.
- * @return List of region locations
- * @throws IOException, InterruptedException
- */
+  /**
+   * Splits a region
+   * @param t Region to split.
+   * @return List of region locations
+   */
   private List<HRegionLocation> splitRegion(final RegionInfo r)
-      throws IOException, InterruptedException {
+      throws IOException, InterruptedException, ExecutionException {
     List<HRegionLocation> locations = new ArrayList<>();
     // Split this table in two.
     Admin admin = TEST_UTIL.getAdmin();
     Connection connection = TEST_UTIL.getConnection();
-    admin.splitRegion(r.getEncodedNameAsBytes());
+    admin.splitRegionAsync(r.getEncodedNameAsBytes(), null).get();
     admin.close();
     PairOfSameType<RegionInfo> regions = waitOnDaughters(r);
     if (regions != null) {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterBalanceThrottling.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterBalanceThrottling.java
index 80f0fb6..93cd4f4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterBalanceThrottling.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterBalanceThrottling.java
@@ -25,8 +25,8 @@ import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -147,7 +147,7 @@ public class TestMasterBalanceThrottling {
       Thread.sleep(100);
     }
     HRegionServer biasedServer = TEST_UTIL.getMiniHBaseCluster().getRegionServer(0);
-    for (HRegionInfo regionInfo : TEST_UTIL.getAdmin().getTableRegions(tableName)) {
+    for (RegionInfo regionInfo : TEST_UTIL.getAdmin().getRegions(tableName)) {
       master.move(regionInfo.getEncodedNameAsBytes(),
         Bytes.toBytes(biasedServer.getServerName().getServerName()));
     }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java
index 4d5c946..44cf1cc 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java
@@ -219,9 +219,9 @@ public class TestMasterOperationsForRegionReplicas {
       assert(ADMIN.isTableDisabled(tableName));
       //increase the replica
       desc.setRegionReplication(numReplica + 1);
-      ADMIN.modifyTable(tableName, desc);
+      ADMIN.modifyTable(desc);
       ADMIN.enableTable(tableName);
-      LOG.info(ADMIN.getTableDescriptor(tableName).toString());
+      LOG.info(ADMIN.getDescriptor(tableName).toString());
       assert(ADMIN.isTableEnabled(tableName));
       List<RegionInfo> regions = TEST_UTIL.getMiniHBaseCluster().getMaster().
           getAssignmentManager().getRegionStates().getRegionsOfTable(tableName);
@@ -231,7 +231,7 @@ public class TestMasterOperationsForRegionReplicas {
       //decrease the replica(earlier, table was modified to have a replica count of numReplica + 1)
       ADMIN.disableTable(tableName);
       desc.setRegionReplication(numReplica);
-      ADMIN.modifyTable(tableName, desc);
+      ADMIN.modifyTable(desc);
       ADMIN.enableTable(tableName);
       assert(ADMIN.isTableEnabled(tableName));
       regions = TEST_UTIL.getMiniHBaseCluster().getMaster()
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterStatusServlet.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterStatusServlet.java
index cf80d8e..9f377a0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterStatusServlet.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterStatusServlet.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.master;
 import java.io.IOException;
 import java.io.StringWriter;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
@@ -31,6 +32,7 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 import org.apache.hadoop.hbase.master.assignment.RegionStates;
 import org.apache.hadoop.hbase.regionserver.MetricsRegionServer;
@@ -118,11 +120,9 @@ public class TestMasterStatusServlet {
   }
 
   private void setupMockTables() throws IOException {
-    HTableDescriptor tables[] = new HTableDescriptor[] {
-        new HTableDescriptor(TableName.valueOf("foo")),
-        new HTableDescriptor(TableName.valueOf("bar"))
-    };
-    Mockito.doReturn(tables).when(admin).listTables();
+    List<TableDescriptor> tables = Arrays.asList(new HTableDescriptor(TableName.valueOf("foo")),
+      new HTableDescriptor(TableName.valueOf("bar")));
+    Mockito.doReturn(tables).when(admin).listTableDescriptors();
   }
 
   @Test
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentOnRSCrash.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentOnRSCrash.java
index c4055fe..839d611 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentOnRSCrash.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentOnRSCrash.java
@@ -106,7 +106,7 @@ public class TestAssignmentOnRSCrash {
       throws Exception {
     final int NROWS = 100;
     int nkilled = 0;
-    for (RegionInfo hri: UTIL.getHBaseAdmin().getTableRegions(TEST_TABLE)) {
+    for (RegionInfo hri: UTIL.getHBaseAdmin().getRegions(TEST_TABLE)) {
       ServerName serverName = AssignmentTestingUtil.getServerHoldingRegion(UTIL, hri);
       if (AssignmentTestingUtil.isServerHoldingMeta(UTIL, serverName)) continue;
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRogueRSAssignment.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRogueRSAssignment.java
index c88e583..70388ef 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRogueRSAssignment.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRogueRSAssignment.java
@@ -26,13 +26,13 @@ import java.util.List;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.YouAreDeadException;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.master.HMaster;
@@ -125,7 +125,7 @@ public class TestRogueRSAssignment {
   @Before
   public void setup() throws IOException {
     // Turn off balancer
-    admin.setBalancerRunning(false, true);
+    admin.balancerSwitch(false, true);
   }
 
   @After
@@ -135,7 +135,7 @@ public class TestRogueRSAssignment {
       UTIL.deleteTable(td.getTableName());
     }
     // Turn on balancer
-    admin.setBalancerRunning(true, false);
+    admin.balancerSwitch(true, false);
   }
 
   /**
@@ -146,7 +146,7 @@ public class TestRogueRSAssignment {
   public void testReportRSWithWrongRegion() throws Exception {
     final TableName tableName = TableName.valueOf(this.name.getMethodName());
 
-    List<HRegionInfo> tableRegions = createTable(tableName);
+    List<RegionInfo> tableRegions = createTable(tableName);
 
     final ServerName sn = ServerName.parseVersionedServerName(
         ServerName.valueOf("1.example.org", 1, System.currentTimeMillis()).getVersionedBytes());
@@ -164,7 +164,7 @@ public class TestRogueRSAssignment {
   }
 
   private RegionServerStatusProtos.RegionServerReportRequest.Builder
-      makeRSReportRequestWithRegions(final ServerName sn, HRegionInfo... regions) {
+      makeRSReportRequestWithRegions(final ServerName sn, RegionInfo... regions) {
     ClusterStatusProtos.ServerLoad.Builder sl = ClusterStatusProtos.ServerLoad.newBuilder();
     for (int i = 0; i < regions.length; i++) {
       HBaseProtos.RegionSpecifier.Builder rs = HBaseProtos.RegionSpecifier.newBuilder();
@@ -182,7 +182,7 @@ public class TestRogueRSAssignment {
               .setLoad(sl);
   }
 
-  private List<HRegionInfo> createTable(final TableName tableName) throws Exception {
+  private List<RegionInfo> createTable(final TableName tableName) throws Exception {
     TableDescriptorBuilder tdBuilder = TableDescriptorBuilder.newBuilder(tableName);
     tdBuilder.setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILY).build());
 
@@ -194,10 +194,10 @@ public class TestRogueRSAssignment {
     return assertRegionCount(tableName, initialRegionCount);
   }
 
-  private List<HRegionInfo> assertRegionCount(final TableName tableName, final int nregions)
+  private List<RegionInfo> assertRegionCount(final TableName tableName, final int nregions)
       throws Exception {
     UTIL.waitUntilNoRegionsInTransition();
-    List<HRegionInfo> tableRegions = admin.getTableRegions(tableName);
+    List<RegionInfo> tableRegions = admin.getRegions(tableName);
     assertEquals(nregions, tableRegions.size());
     return tableRegions;
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredNodeTableImport.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredNodeTableImport.java
index b175bd7..612b7ef 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredNodeTableImport.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredNodeTableImport.java
@@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.master.balancer;
 
 import static org.apache.hadoop.hbase.favored.FavoredNodeAssignmentHelper.FAVORED_NODES_NUM;
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 
 import java.util.List;
@@ -29,11 +28,11 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.favored.FavoredNodesManager;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -82,7 +81,7 @@ public class TestFavoredNodeTableImport {
       Threads.sleep(1);
     }
     Admin admin = UTIL.getAdmin();
-    admin.setBalancerRunning(false, true);
+    admin.balancerSwitch(false, true);
 
     String tableName = "testFNImport";
     HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
@@ -107,8 +106,8 @@ public class TestFavoredNodeTableImport {
 
     FavoredNodesManager fnm = UTIL.getHBaseCluster().getMaster().getFavoredNodesManager();
 
-    List<HRegionInfo> regionsOfTable = admin.getTableRegions(TableName.valueOf(tableName));
-    for (HRegionInfo rInfo : regionsOfTable) {
+    List<RegionInfo> regionsOfTable = admin.getRegions(TableName.valueOf(tableName));
+    for (RegionInfo rInfo : regionsOfTable) {
       Set<ServerName> favNodes = Sets.newHashSet(fnm.getFavoredNodes(rInfo));
       assertNotNull(favNodes);
       assertEquals("Required no of favored nodes not found.", FAVORED_NODES_NUM, favNodes.size());
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticBalancerPickers.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticBalancerPickers.java
index 0b40ae0..6e1c84d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticBalancerPickers.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticBalancerPickers.java
@@ -109,7 +109,7 @@ public class TestFavoredStochasticBalancerPickers extends BalancerTestBase {
     TEST_UTIL.getHBaseCluster().waitForActiveAndReadyMaster(120*1000);
     cluster = TEST_UTIL.getMiniHBaseCluster();
     admin = TEST_UTIL.getAdmin();
-    admin.setBalancerRunning(false, true);
+    admin.balancerSwitch(false, true);
   }
 
   @After
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java
index 4bc566c..494fca3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java
@@ -104,7 +104,7 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
     cluster = TEST_UTIL.getMiniHBaseCluster();
     master = TEST_UTIL.getMiniHBaseCluster().getMaster();
     admin = TEST_UTIL.getAdmin();
-    admin.setBalancerRunning(false, true);
+    admin.balancerSwitch(false, true);
   }
 
   @After
@@ -130,8 +130,8 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
 
     // Now try to run balance, and verify no regions are moved to the 2 region servers recently
     // started.
-    admin.setBalancerRunning(true, true);
-    assertTrue("Balancer did not run", admin.balancer());
+    admin.balancerSwitch(true, true);
+    assertTrue("Balancer did not run", admin.balance());
     TEST_UTIL.waitUntilNoRegionsInTransition(120000);
 
     List<RegionInfo> hris = admin.getRegions(rs1.getRegionServer().getServerName());
@@ -159,7 +159,7 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
 
     LoadBalancer balancer = master.getLoadBalancer();
     List<RegionInfo> regions = admin.getRegions(tableName);
-    regions.addAll(admin.getTableRegions(TableName.META_TABLE_NAME));
+    regions.addAll(admin.getRegions(TableName.META_TABLE_NAME));
     List<ServerName> servers = Lists.newArrayList(
       admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)).getLiveServerMetrics().keySet());
     Map<ServerName, List<RegionInfo>> map = balancer.roundRobinAssignment(regions, servers);
@@ -219,7 +219,7 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
     admin.createTable(desc);
     TEST_UTIL.waitTableAvailable(desc.getTableName());
 
-    RegionInfo hri = admin.getTableRegions(TableName.valueOf(tableName)).get(0);
+    RegionInfo hri = admin.getRegions(TableName.valueOf(tableName)).get(0);
 
     FavoredNodesManager fnm = master.getFavoredNodesManager();
     fnm.deleteFavoredNodesForRegions(Lists.newArrayList(hri));
@@ -250,7 +250,7 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
     admin.createTable(desc, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM);
     TEST_UTIL.waitTableAvailable(tableName);
 
-    final RegionInfo region = admin.getTableRegions(tableName).get(0);
+    final RegionInfo region = admin.getRegions(tableName).get(0);
     LOG.info("Region thats supposed to be in transition: " + region);
     FavoredNodesManager fnm = master.getFavoredNodesManager();
     List<ServerName> currentFN = fnm.getFavoredNodes(region);
@@ -259,10 +259,10 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
     fnm.deleteFavoredNodesForRegions(Lists.newArrayList(region));
 
     RegionStates regionStates = master.getAssignmentManager().getRegionStates();
-    admin.setBalancerRunning(true, true);
+    admin.balancerSwitch(true, true);
 
     // Balancer should unassign the region
-    assertTrue("Balancer did not run", admin.balancer());
+    assertTrue("Balancer did not run", admin.balance());
     TEST_UTIL.waitUntilNoRegionsInTransition();
 
     admin.assign(region.getEncodedNameAsBytes());
@@ -273,7 +273,7 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
     assertEquals("Expected number of FN not present",
       FavoredNodeAssignmentHelper.FAVORED_NODES_NUM, currentFN.size());
 
-    assertTrue("Balancer did not run", admin.balancer());
+    assertTrue("Balancer did not run", admin.balance());
     TEST_UTIL.waitUntilNoRegionsInTransition(60000);
 
     checkFavoredNodeAssignments(tableName, fnm, regionStates);
@@ -288,7 +288,7 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
     admin.createTable(desc, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM);
     TEST_UTIL.waitTableAvailable(tableName);
 
-    final RegionInfo misplacedRegion = admin.getTableRegions(tableName).get(0);
+    final RegionInfo misplacedRegion = admin.getRegions(tableName).get(0);
     FavoredNodesManager fnm = master.getFavoredNodesManager();
     List<ServerName> currentFN = fnm.getFavoredNodes(misplacedRegion);
     assertNotNull(currentFN);
@@ -314,8 +314,8 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
     final ServerName current = regionStates.getRegionServerOfRegion(misplacedRegion);
     assertNull("Misplaced region is still hosted on favored node, not expected.",
         FavoredNodesPlan.getFavoredServerPosition(fnm.getFavoredNodes(misplacedRegion), current));
-    admin.setBalancerRunning(true, true);
-    assertTrue("Balancer did not run", admin.balancer());
+    admin.balancerSwitch(true, true);
+    assertTrue("Balancer did not run", admin.balance());
     TEST_UTIL.waitFor(120000, 30000, new Waiter.Predicate<Exception>() {
       @Override
       public boolean evaluate() throws Exception {
@@ -335,7 +335,7 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
     admin.createTable(desc, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM);
     TEST_UTIL.waitTableAvailable(tableName);
 
-    final RegionInfo region = admin.getTableRegions(tableName).get(0);
+    final RegionInfo region = admin.getRegions(tableName).get(0);
     LOG.info("Region that's supposed to be in transition: " + region);
     FavoredNodesManager fnm = master.getFavoredNodesManager();
     List<ServerName> currentFN = fnm.getFavoredNodes(region);
@@ -356,9 +356,9 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
       }
     });
 
-    assertEquals("Not all regions are online", REGION_NUM, admin.getTableRegions(tableName).size());
-    admin.setBalancerRunning(true, true);
-    assertTrue("Balancer did not run", admin.balancer());
+    assertEquals("Not all regions are online", REGION_NUM, admin.getRegions(tableName).size());
+    admin.balancerSwitch(true, true);
+    assertTrue("Balancer did not run", admin.balance());
     TEST_UTIL.waitUntilNoRegionsInTransition(60000);
 
     checkFavoredNodeAssignments(tableName, fnm, regionStates);
@@ -373,7 +373,7 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
     admin.createTable(desc, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM);
     TEST_UTIL.waitTableAvailable(tableName);
 
-    final RegionInfo region = admin.getTableRegions(tableName).get(0);
+    final RegionInfo region = admin.getRegions(tableName).get(0);
     LOG.info("Region that's supposed to be in transition: " + region);
     FavoredNodesManager fnm = master.getFavoredNodesManager();
     List<ServerName> currentFN = fnm.getFavoredNodes(region);
@@ -417,10 +417,10 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
       admin.assign(regionInfo.getEncodedNameAsBytes());
     }
     TEST_UTIL.waitUntilNoRegionsInTransition(60000);
-    assertEquals("Not all regions are online", REGION_NUM, admin.getTableRegions(tableName).size());
+    assertEquals("Not all regions are online", REGION_NUM, admin.getRegions(tableName).size());
 
-    admin.setBalancerRunning(true, true);
-    assertTrue("Balancer did not run", admin.balancer());
+    admin.balancerSwitch(true, true);
+    assertTrue("Balancer did not run", admin.balance());
     TEST_UTIL.waitUntilNoRegionsInTransition(60000);
 
     checkFavoredNodeAssignments(tableName, fnm, regionStates);
@@ -435,7 +435,7 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
     admin.createTable(desc, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM);
     TEST_UTIL.waitTableAvailable(tableName);
 
-    final RegionInfo region = admin.getTableRegions(tableName).get(0);
+    final RegionInfo region = admin.getRegions(tableName).get(0);
     LOG.info("Region that's supposed to be in transition: " + region);
     FavoredNodesManager fnm = master.getFavoredNodesManager();
     List<ServerName> currentFN = fnm.getFavoredNodes(region);
@@ -511,10 +511,10 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
       admin.assign(regionInfo.getEncodedNameAsBytes());
     }
     TEST_UTIL.waitUntilNoRegionsInTransition(60000);
-    assertEquals("Not all regions are online", REGION_NUM, admin.getTableRegions(tableName).size());
+    assertEquals("Not all regions are online", REGION_NUM, admin.getRegions(tableName).size());
 
-    admin.setBalancerRunning(true, true);
-    assertTrue("Balancer did not run", admin.balancer());
+    admin.balancerSwitch(true, true);
+    assertTrue("Balancer did not run", admin.balance());
     TEST_UTIL.waitUntilNoRegionsInTransition(60000);
 
     checkFavoredNodeAssignments(tableName, fnm, regionStates);
@@ -522,7 +522,7 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
 
   private void checkFavoredNodeAssignments(TableName tableName, FavoredNodesManager fnm,
       RegionStates regionStates) throws IOException {
-    for (RegionInfo hri : admin.getTableRegions(tableName)) {
+    for (RegionInfo hri : admin.getRegions(tableName)) {
       ServerName host = regionStates.getRegionServerOfRegion(hri);
       assertNotNull("Region: " + hri.getEncodedName() + " not on FN, current: " + host
               + " FN list: " + fnm.getFavoredNodes(hri),
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/locking/TestLockManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/locking/TestLockManager.java
index d04ef79..6860b7a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/locking/TestLockManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/locking/TestLockManager.java
@@ -25,9 +25,9 @@ import java.util.List;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
@@ -67,7 +67,7 @@ public class TestLockManager {
 
   private static String namespace = "namespace";
   private static TableName tableName = TableName.valueOf(namespace, "table");
-  private static HRegionInfo[] tableRegions;
+  private static RegionInfo[] tableRegions;
 
   private static void setupConf(Configuration conf) {
     conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1);
@@ -83,9 +83,9 @@ public class TestLockManager {
     UTIL.getAdmin().createNamespace(NamespaceDescriptor.create(namespace).build());
     UTIL.createTable(tableName, new byte[][]{Bytes.toBytes("fam")},
         new byte[][] {Bytes.toBytes("1")});
-    List<HRegionInfo> regions = UTIL.getAdmin().getTableRegions(tableName);
+    List<RegionInfo> regions = UTIL.getAdmin().getRegions(tableName);
     assert regions.size() > 0;
-    tableRegions = new HRegionInfo[regions.size()];
+    tableRegions = new RegionInfo[regions.size()];
     regions.toArray(tableRegions);
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java
index 6f7f69e..537090a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java
@@ -141,9 +141,9 @@ public class TestSimpleRegionNormalizerOnCluster {
       region.flush(true);
     }
 
-    HTableDescriptor htd = new HTableDescriptor(admin.getTableDescriptor(TABLENAME));
+    HTableDescriptor htd = new HTableDescriptor(admin.getDescriptor(TABLENAME));
     htd.setNormalizationEnabled(true);
-    admin.modifyTable(TABLENAME, htd);
+    admin.modifyTable(htd);
 
     admin.flush(TABLENAME);
 
@@ -213,9 +213,9 @@ public class TestSimpleRegionNormalizerOnCluster {
       region.flush(true);
     }
 
-    HTableDescriptor htd = new HTableDescriptor(admin.getTableDescriptor(tableName));
+    HTableDescriptor htd = new HTableDescriptor(admin.getDescriptor(tableName));
     htd.setNormalizationEnabled(true);
-    admin.modifyTable(tableName, htd);
+    admin.modifyTable(htd);
 
     admin.flush(tableName);
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCloneSnapshotProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCloneSnapshotProcedure.java
index f4fa282..d90192a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCloneSnapshotProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCloneSnapshotProcedure.java
@@ -71,7 +71,7 @@ public class TestCloneSnapshotProcedure extends TestTableDDLProcedureBase {
     if (snapshot == null) {
       final TableName snapshotTableName = TableName.valueOf("testCloneSnapshot");
       long tid = System.currentTimeMillis();
-      final byte[] snapshotName = Bytes.toBytes("snapshot-" + tid);
+      final String snapshotName = "snapshot-" + tid;
 
       Admin admin = UTIL.getAdmin();
       // create Table
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteColumnFamilyProcedureFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteColumnFamilyProcedureFromClient.java
index d5ec62d..91295c5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteColumnFamilyProcedureFromClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteColumnFamilyProcedureFromClient.java
@@ -103,7 +103,7 @@ public class TestDeleteColumnFamilyProcedureFromClient {
   @Test
   public void deleteColumnFamilyWithMultipleRegions() throws Exception {
     Admin admin = TEST_UTIL.getAdmin();
-    HTableDescriptor beforehtd = admin.getTableDescriptor(TABLENAME);
+    HTableDescriptor beforehtd = new HTableDescriptor(admin.getDescriptor(TABLENAME));
 
     FileSystem fs = TEST_UTIL.getDFSCluster().getFileSystem();
 
@@ -150,7 +150,7 @@ public class TestDeleteColumnFamilyProcedureFromClient {
     admin.deleteColumnFamily(TABLENAME, Bytes.toBytes("cf2"));
 
     // 5 - Check if only 2 column families exist in the descriptor
-    HTableDescriptor afterhtd = admin.getTableDescriptor(TABLENAME);
+    HTableDescriptor afterhtd = new HTableDescriptor(admin.getDescriptor(TABLENAME));
     assertEquals(2, afterhtd.getColumnFamilyCount());
     HColumnDescriptor[] newFamilies = afterhtd.getColumnFamilies();
     assertTrue(newFamilies[0].getNameAsString().equals("cf1"));
@@ -181,7 +181,7 @@ public class TestDeleteColumnFamilyProcedureFromClient {
   @Test
   public void deleteColumnFamilyTwice() throws Exception {
     Admin admin = TEST_UTIL.getAdmin();
-    HTableDescriptor beforehtd = admin.getTableDescriptor(TABLENAME);
+    HTableDescriptor beforehtd = new HTableDescriptor(admin.getDescriptor(TABLENAME));
     String cfToDelete = "cf1";
 
     FileSystem fs = TEST_UTIL.getDFSCluster().getFileSystem();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterFailoverWithProcedures.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterFailoverWithProcedures.java
index 5e574b5..ef65f2f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterFailoverWithProcedures.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterFailoverWithProcedures.java
@@ -205,7 +205,7 @@ public class TestMasterFailoverWithProcedures {
     UTIL.waitUntilAllRegionsAssigned(tableName);
 
     // validate the table regions and layout
-    regions = UTIL.getAdmin().getTableRegions(tableName).toArray(new RegionInfo[0]);
+    regions = UTIL.getAdmin().getRegions(tableName).toArray(new RegionInfo[0]);
     if (preserveSplits) {
       assertEquals(1 + splitKeys.length, regions.length);
     } else {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyTableProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyTableProcedure.java
index a2dccf8..8bd3958 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyTableProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyTableProcedure.java
@@ -73,7 +73,7 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
     UTIL.getAdmin().disableTable(tableName);
 
     // Modify the table descriptor
-    HTableDescriptor htd = new HTableDescriptor(UTIL.getAdmin().getTableDescriptor(tableName));
+    HTableDescriptor htd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
 
     // Test 1: Modify 1 property
     long newMaxFileSize = htd.getMaxFileSize() * 2;
@@ -84,7 +84,7 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
         procExec, new ModifyTableProcedure(procExec.getEnvironment(), htd));
     ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId1));
 
-    HTableDescriptor currentHtd = UTIL.getAdmin().getTableDescriptor(tableName);
+    HTableDescriptor currentHtd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
     assertEquals(newMaxFileSize, currentHtd.getMaxFileSize());
 
     // Test 2: Modify multiple properties
@@ -97,7 +97,7 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
         procExec, new ModifyTableProcedure(procExec.getEnvironment(), htd));
     ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId2));
 
-    currentHtd = UTIL.getAdmin().getTableDescriptor(tableName);
+    currentHtd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
     assertEquals(newReadOnlyOption, currentHtd.isReadOnly());
     assertEquals(newMemStoreFlushSize, currentHtd.getMemStoreFlushSize());
   }
@@ -108,19 +108,19 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
     final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
 
     MasterProcedureTestingUtility.createTable(procExec, tableName, null, "cf1");
-    HTableDescriptor currentHtd = UTIL.getAdmin().getTableDescriptor(tableName);
+    HTableDescriptor currentHtd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
     assertEquals(1, currentHtd.getFamiliesKeys().size());
 
     // Test 1: Modify the table descriptor online
     String cf2 = "cf2";
-    HTableDescriptor htd = new HTableDescriptor(UTIL.getAdmin().getTableDescriptor(tableName));
+    HTableDescriptor htd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
     htd.addFamily(new HColumnDescriptor(cf2));
 
     long procId = ProcedureTestingUtility.submitAndWait(
         procExec, new ModifyTableProcedure(procExec.getEnvironment(), htd));
     ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId));
 
-    currentHtd = UTIL.getAdmin().getTableDescriptor(tableName);
+    currentHtd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
     assertEquals(2, currentHtd.getFamiliesKeys().size());
     assertTrue(currentHtd.hasFamily(Bytes.toBytes(cf2)));
 
@@ -129,7 +129,7 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
     ProcedureTestingUtility.waitNoProcedureRunning(procExec);
     String cf3 = "cf3";
     HTableDescriptor htd2 =
-        new HTableDescriptor(UTIL.getAdmin().getTableDescriptor(tableName));
+        new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
     htd2.addFamily(new HColumnDescriptor(cf3));
 
     long procId2 =
@@ -137,7 +137,7 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
           new ModifyTableProcedure(procExec.getEnvironment(), htd2));
     ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId2));
 
-    currentHtd = UTIL.getAdmin().getTableDescriptor(tableName);
+    currentHtd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
     assertTrue(currentHtd.hasFamily(Bytes.toBytes(cf3)));
     assertEquals(3, currentHtd.getFamiliesKeys().size());
   }
@@ -151,18 +151,18 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
     final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
 
     MasterProcedureTestingUtility.createTable(procExec, tableName, null, cf1, cf2, cf3);
-    HTableDescriptor currentHtd = UTIL.getAdmin().getTableDescriptor(tableName);
+    HTableDescriptor currentHtd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
     assertEquals(3, currentHtd.getFamiliesKeys().size());
 
     // Test 1: Modify the table descriptor
-    HTableDescriptor htd = new HTableDescriptor(UTIL.getAdmin().getTableDescriptor(tableName));
+    HTableDescriptor htd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
     htd.removeFamily(Bytes.toBytes(cf2));
 
     long procId = ProcedureTestingUtility.submitAndWait(
         procExec, new ModifyTableProcedure(procExec.getEnvironment(), htd));
     ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId));
 
-    currentHtd = UTIL.getAdmin().getTableDescriptor(tableName);
+    currentHtd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
     assertEquals(2, currentHtd.getFamiliesKeys().size());
     assertFalse(currentHtd.hasFamily(Bytes.toBytes(cf2)));
 
@@ -171,7 +171,7 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
     ProcedureTestingUtility.waitNoProcedureRunning(procExec);
 
     HTableDescriptor htd2 =
-        new HTableDescriptor(UTIL.getAdmin().getTableDescriptor(tableName));
+        new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
     htd2.removeFamily(Bytes.toBytes(cf3));
     // Disable Sanity check
     htd2.setConfiguration("hbase.table.sanity.checks", Boolean.FALSE.toString());
@@ -181,13 +181,13 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
           new ModifyTableProcedure(procExec.getEnvironment(), htd2));
     ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId2));
 
-    currentHtd = UTIL.getAdmin().getTableDescriptor(tableName);
+    currentHtd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
     assertEquals(1, currentHtd.getFamiliesKeys().size());
     assertFalse(currentHtd.hasFamily(Bytes.toBytes(cf3)));
 
     //Removing the last family will fail
     HTableDescriptor htd3 =
-        new HTableDescriptor(UTIL.getAdmin().getTableDescriptor(tableName));
+        new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
     htd3.removeFamily(Bytes.toBytes(cf1));
     long procId3 =
         ProcedureTestingUtility.submitAndWait(procExec,
@@ -256,7 +256,7 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
     ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
 
     // Modify multiple properties of the table.
-    HTableDescriptor htd = new HTableDescriptor(UTIL.getAdmin().getTableDescriptor(tableName));
+    HTableDescriptor htd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
     boolean newCompactionEnableOption = htd.isCompactionEnabled() ? false : true;
     htd.setCompactionEnabled(newCompactionEnableOption);
     htd.addFamily(new HColumnDescriptor(cf2));
@@ -270,7 +270,7 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
     MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId);
 
     // Validate descriptor
-    HTableDescriptor currentHtd = UTIL.getAdmin().getTableDescriptor(tableName);
+    HTableDescriptor currentHtd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
     assertEquals(newCompactionEnableOption, currentHtd.isCompactionEnabled());
     assertEquals(2, currentHtd.getFamiliesKeys().size());
     assertTrue(currentHtd.hasFamily(Bytes.toBytes(cf2)));
@@ -554,7 +554,7 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
 
       public void run() {
         try {
-          UTIL.getAdmin().modifyTable(tableName, htd);
+          UTIL.getAdmin().modifyTable(htd);
         } catch (Exception e) {
           if (e.getClass().equals(ConcurrentTableModificationException.class)) {
             this.exception = true;
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedureAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedureAdmin.java
index 02f0257..c54aea4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedureAdmin.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedureAdmin.java
@@ -25,9 +25,9 @@ import java.util.Random;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
@@ -88,7 +88,7 @@ public class TestProcedureAdmin {
   public void tearDown() throws Exception {
     assertTrue("expected executor to be running", getMasterProcedureExecutor().isRunning());
     ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(getMasterProcedureExecutor(), false);
-    for (HTableDescriptor htd: UTIL.getAdmin().listTables()) {
+    for (TableDescriptor htd: UTIL.getAdmin().listTableDescriptors()) {
       LOG.info("Tear down, remove table=" + htd.getTableName());
       UTIL.deleteTable(htd.getTableName());
     }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestRestoreSnapshotProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestRestoreSnapshotProcedure.java
index e9e5ebe..0f7b7ca 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestRestoreSnapshotProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestRestoreSnapshotProcedure.java
@@ -99,7 +99,7 @@ public class TestRestoreSnapshotProcedure extends TestTableDDLProcedureBase {
 
   private void setupSnapshotAndUpdateTable() throws Exception {
     long tid = System.currentTimeMillis();
-    final byte[] snapshotName = Bytes.toBytes("snapshot-" + tid);
+    final String snapshotName = "snapshot-" + tid;
     Admin admin = UTIL.getAdmin();
     // create Table
     SnapshotTestingUtils.createTable(UTIL, snapshotTableName, getNumReplicas(), CF1, CF2);
@@ -108,7 +108,7 @@ public class TestRestoreSnapshotProcedure extends TestTableDDLProcedureBase {
     SnapshotTestingUtils.loadData(UTIL, snapshotTableName, rowCountCF2, CF2);
     SnapshotTestingUtils.verifyRowCount(UTIL, snapshotTableName, rowCountCF1 + rowCountCF2);
 
-    snapshotHTD = admin.getTableDescriptor(snapshotTableName);
+    snapshotHTD = new HTableDescriptor(admin.getDescriptor(snapshotTableName));
 
     admin.disableTable(snapshotTableName);
     // take a snapshot
@@ -128,7 +128,7 @@ public class TestRestoreSnapshotProcedure extends TestTableDDLProcedureBase {
     SnapshotTestingUtils.loadData(UTIL, snapshotTableName, rowCountCF3, CF3);
     SnapshotTestingUtils.loadData(UTIL, snapshotTableName, rowCountCF4, CF4);
     SnapshotTestingUtils.loadData(UTIL, snapshotTableName, rowCountCF1addition, CF1);
-    HTableDescriptor currentHTD = admin.getTableDescriptor(snapshotTableName);
+    HTableDescriptor currentHTD = new HTableDescriptor(admin.getDescriptor(snapshotTableName));
     assertTrue(currentHTD.hasFamily(CF1));
     assertFalse(currentHTD.hasFamily(CF2));
     assertTrue(currentHTD.hasFamily(CF3));
@@ -216,7 +216,8 @@ public class TestRestoreSnapshotProcedure extends TestTableDDLProcedureBase {
     try {
       UTIL.getAdmin().enableTable(snapshotTableName);
 
-      HTableDescriptor currentHTD = UTIL.getAdmin().getTableDescriptor(snapshotTableName);
+      HTableDescriptor currentHTD =
+        new HTableDescriptor(UTIL.getAdmin().getDescriptor(snapshotTableName));
       assertTrue(currentHTD.hasFamily(CF1));
       assertTrue(currentHTD.hasFamily(CF2));
       assertFalse(currentHTD.hasFamily(CF3));
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableDDLProcedureBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableDDLProcedureBase.java
index f7cf640..bd7f26d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableDDLProcedureBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableDDLProcedureBase.java
@@ -18,9 +18,11 @@
 
 package org.apache.hadoop.hbase.master.procedure;
 
+import static org.junit.Assert.assertTrue;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
@@ -31,8 +33,6 @@ import org.junit.BeforeClass;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.junit.Assert.assertTrue;
-
 public abstract class TestTableDDLProcedureBase {
   private static final Logger LOG = LoggerFactory.getLogger(TestTableDDLProcedureBase.class);
   protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
@@ -64,7 +64,7 @@ public abstract class TestTableDDLProcedureBase {
   @After
   public void tearDown() throws Exception {
     resetProcExecutorTestingKillFlag();
-    for (HTableDescriptor htd: UTIL.getAdmin().listTables()) {
+    for (TableDescriptor htd : UTIL.getAdmin().listTableDescriptors()) {
       LOG.info("Tear down, remove table=" + htd.getTableName());
       UTIL.deleteTable(htd.getTableName());
     }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableDescriptorModificationFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableDescriptorModificationFromClient.java
index d3b13ad..11d1f19 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableDescriptorModificationFromClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableDescriptorModificationFromClient.java
@@ -101,7 +101,7 @@ public class TestTableDescriptorModificationFromClient {
       HTableDescriptor modifiedHtd = new HTableDescriptor(TABLE_NAME);
       modifiedHtd.addFamily(new HColumnDescriptor(FAMILY_0));
       modifiedHtd.addFamily(new HColumnDescriptor(FAMILY_1));
-      admin.modifyTable(TABLE_NAME, modifiedHtd);
+      admin.modifyTable(modifiedHtd);
       verifyTableDescriptor(TABLE_NAME, FAMILY_0, FAMILY_1);
     } finally {
       admin.deleteTable(TABLE_NAME);
@@ -178,7 +178,7 @@ public class TestTableDescriptorModificationFromClient {
       // Modify colymn family
       admin.modifyColumnFamily(TABLE_NAME, cfDescriptor);
 
-      HTableDescriptor htd = admin.getTableDescriptor(TABLE_NAME);
+      HTableDescriptor htd = new HTableDescriptor(admin.getDescriptor(TABLE_NAME));
       HColumnDescriptor hcfd = htd.getFamily(FAMILY_0);
       assertTrue(hcfd.getBlocksize() == newBlockSize);
     } finally {
@@ -267,12 +267,12 @@ public class TestTableDescriptorModificationFromClient {
     }
   }
 
-  private void verifyTableDescriptor(final TableName tableName,
-                                     final byte[]... families) throws IOException {
+  private void verifyTableDescriptor(final TableName tableName, final byte[]... families)
+      throws IOException {
     Admin admin = TEST_UTIL.getAdmin();
 
     // Verify descriptor from master
-    HTableDescriptor htd = admin.getTableDescriptor(tableName);
+    HTableDescriptor htd = new HTableDescriptor(admin.getDescriptor(tableName));
     verifyTableDescriptor(htd, tableName, families);
 
     // Verify descriptor from HDFS
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestMobCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestMobCompactor.java
index 86e07d2..130a6d3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestMobCompactor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestMobCompactor.java
@@ -92,7 +92,6 @@ import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.Threads;
 import org.junit.AfterClass;
 import org.junit.Assert;
@@ -268,13 +267,7 @@ public class TestMobCompactor {
 
     hcd1.setMobCompactPartitionPolicy(type);
     desc.modifyFamily(hcd1);
-    admin.modifyTable(tableName, desc);
-    Pair<Integer, Integer> st;
-
-    while (null != (st = admin.getAlterStatus(tableName)) && st.getFirst() > 0) {
-      LOG.debug(st.getFirst() + " regions left to update");
-      Thread.sleep(40);
-    }
+    admin.modifyTable(desc);
     LOG.info("alter status finished");
   }
 
@@ -439,7 +432,7 @@ public class TestMobCompactor {
     int rowNumPerRegion = count * rowNumPerFile;
 
     long tid = System.currentTimeMillis();
-    byte[] snapshotName1 = Bytes.toBytes("snaptb-" + tid);
+    String snapshotName1 = "snaptb-" + tid;
     // take a snapshot
     admin.snapshot(snapshotName1, tableName);
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
index a5d12b7..7b32b5c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
@@ -42,13 +42,11 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Waiter;
-import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.CompactionState;
 import org.apache.hadoop.hbase.client.Connection;
@@ -83,8 +81,6 @@ import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.hadoop.hbase.util.RetryCounter;
-import org.apache.hadoop.hbase.util.Threads;
 import org.apache.zookeeper.KeeperException;
 import org.junit.After;
 import org.junit.AfterClass;
@@ -130,7 +126,7 @@ public class TestNamespaceAuditor {
 
   @After
   public void cleanup() throws Exception, KeeperException {
-    for (HTableDescriptor table : ADMIN.listTables()) {
+    for (TableDescriptor table : ADMIN.listTableDescriptors()) {
       ADMIN.disableTable(table.getTableName());
       deleteTable(table.getTableName());
     }
@@ -714,8 +710,8 @@ public class TestNamespaceAuditor {
     String snapshot = "snapshot_testRestoreSnapshot";
     ADMIN.snapshot(snapshot, tableName1);
 
-    List<HRegionInfo> regions = ADMIN.getTableRegions(tableName1);
-    Collections.sort(regions);
+    List<RegionInfo> regions = ADMIN.getRegions(tableName1);
+    Collections.sort(regions, RegionInfo.COMPARATOR);
 
     ADMIN.split(tableName1, Bytes.toBytes("JJJ"));
     Thread.sleep(2000);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestProcedureManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestProcedureManager.java
index 1b86b50..8ab9ec8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestProcedureManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestProcedureManager.java
@@ -66,7 +66,7 @@ public class TestProcedureManager {
   public void testSimpleProcedureManager() throws IOException {
     Admin admin = util.getAdmin();
 
-    byte[] result = admin.execProcedureWithRet(SimpleMasterProcedureManager.SIMPLE_SIGNATURE,
+    byte[] result = admin.execProcedureWithReturn(SimpleMasterProcedureManager.SIMPLE_SIGNATURE,
         "mytest", new HashMap<>());
     assertArrayEquals("Incorrect return data from execProcedure",
         Bytes.toBytes(SimpleMasterProcedureManager.SIMPLE_DATA), result);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTablesWithQuotas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTablesWithQuotas.java
index 2298c85..450c554 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTablesWithQuotas.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestTablesWithQuotas.java
@@ -180,7 +180,7 @@ public class TestTablesWithQuotas {
     // Set up Admin to return null (match the implementation)
     Admin admin = mock(Admin.class);
     when(conn.getAdmin()).thenReturn(admin);
-    when(admin.getTableRegions(missingTable)).thenReturn(null);
+    when(admin.getRegions(missingTable)).thenReturn(null);
 
     QuotaObserverChore chore = mock(QuotaObserverChore.class);
     Map<RegionInfo,Long> regionUsage = new HashMap<>();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionFileNotFound.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionFileNotFound.java
index 22a888d..c19adfd 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionFileNotFound.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionFileNotFound.java
@@ -126,12 +126,13 @@ public class TestCompactionFileNotFound {
       }
       table.put(putb);
       HRegion hr1 = (HRegion) util.getRSForFirstRegionInTable(TEST_TABLE)
-          .getRegionByEncodedName(admin.getTableRegions(TEST_TABLE).get(0).getEncodedName());
+          .getRegionByEncodedName(admin.getRegions(TEST_TABLE).get(0).getEncodedName());
       // Refresh store files post compaction, this should not open already compacted files
       hr1.refreshStoreFiles(true);
-      int numRegionsBeforeSplit = admin.getTableRegions(TEST_TABLE).size();
+      int numRegionsBeforeSplit = admin.getRegions(TEST_TABLE).size();
       // Check if we can successfully split after compaction
-      admin.splitRegion(admin.getTableRegions(TEST_TABLE).get(0).getEncodedNameAsBytes(), ROW_C);
+      admin.splitRegionAsync(admin.getRegions(TEST_TABLE).get(0).getEncodedNameAsBytes(), ROW_C)
+        .get();
       util.waitFor(20000, new Waiter.Predicate<Exception>() {
         @Override
         public boolean evaluate() throws Exception {
@@ -181,7 +182,7 @@ public class TestCompactionFileNotFound {
       }
       table.put(putb);
       HRegion hr1 = (HRegion) util.getRSForFirstRegionInTable(TEST_TABLE)
-          .getRegionByEncodedName(admin.getTableRegions(TEST_TABLE).get(0).getEncodedName());
+          .getRegionByEncodedName(admin.getRegions(TEST_TABLE).get(0).getEncodedName());
       // Refresh store files post compaction, this should not open already compacted files
       hr1.refreshStoreFiles(true);
       // Archive the store files and try another compaction to see if all is good
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java
index 1418d6e..57ced95 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java
@@ -262,10 +262,9 @@ public class TestEndToEndSplitTransaction {
 
           log("Initiating region split for:" + region.getRegionNameAsString());
           try {
-            admin.splitRegion(region.getRegionName(), splitPoint);
+            admin.splitRegionAsync(region.getRegionName(), splitPoint).get();
             // wait until the split is complete
             blockUntilRegionSplit(CONF, 50000, region.getRegionName(), true);
-
           } catch (NotServingRegionException ex) {
             // ignore
           }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java
index 9d9226e..ea93468 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java
@@ -197,7 +197,7 @@ public class TestRegionMergeTransactionOnCluster {
   @Test
   public void testCleanMergeReference() throws Exception {
     LOG.info("Starting " + name.getMethodName());
-    ADMIN.enableCatalogJanitor(false);
+    ADMIN.catalogJanitorSwitch(false);
     try {
       final TableName tableName = TableName.valueOf(name.getMethodName());
       // Create table and load data.
@@ -278,7 +278,7 @@ public class TestRegionMergeTransactionOnCluster {
       // files of merging regions
       int cleaned = 0;
       while (cleaned == 0) {
-        cleaned = ADMIN.runCatalogScan();
+        cleaned = ADMIN.runCatalogJanitor();
         LOG.debug("catalog janitor returned " + cleaned);
         Thread.sleep(50);
         // Cleanup is async so wait till all procedures are done running.
@@ -297,7 +297,7 @@ public class TestRegionMergeTransactionOnCluster {
           HConstants.MERGEB_QUALIFIER) != null);
 
     } finally {
-      ADMIN.enableCatalogJanitor(true);
+      ADMIN.catalogJanitorSwitch(true);
     }
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionOpen.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionOpen.java
index 7190d84..7ab5dcc 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionOpen.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionOpen.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.regionserver;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
@@ -37,6 +36,7 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.executor.ExecutorType;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
@@ -125,7 +125,7 @@ public class TestRegionOpen {
     } catch (IOException e) {
       LOG.info("Caught expected IOE due missing .regioninfo file, due: " + e.getMessage() + " skipping region open.");
       // We should only have 1 region online
-      List<HRegionInfo> regions = admin.getTableRegions(tableName);
+      List<RegionInfo> regions = admin.getRegions(tableName);
       LOG.info("Regions: " + regions);
       if (regions.size() != 1) {
         fail("Table " + tableName + " should have only one region, but got more: " + regions);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java
index 40c3d29..3778c20 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java
@@ -154,20 +154,6 @@ public class TestRegionServerMetrics {
     admin.deleteTable(tableName);
   }
 
-  public void waitTableDeleted(TableName name, long timeoutInMillis) throws Exception {
-    long start = System.currentTimeMillis();
-    while (true) {
-      HTableDescriptor[] tables = admin.listTables();
-      for (HTableDescriptor htd : tables) {
-        if (htd.getNameAsString() == name.getNameAsString())
-          return;
-      }
-      if (System.currentTimeMillis() - start > timeoutInMillis)
-        return;
-      Thread.sleep(1000);
-    }
-  }
-
   public void assertCounter(String metric, long expectedValue) {
     metricsHelper.assertCounter(metric, expectedValue, serverSource);
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpoint.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpoint.java
index 04db81a..55da3f4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpoint.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpoint.java
@@ -198,7 +198,7 @@ public class TestRegionReplicaReplicationEndpoint {
 
     HTU.getAdmin().disableTable(htd.getTableName());
     htd.setRegionReplication(2);
-    HTU.getAdmin().modifyTable(htd.getTableName(), htd);
+    HTU.getAdmin().modifyTable(htd);
     HTU.getAdmin().enableTable(htd.getTableName());
 
     // assert peer configuration is correct
@@ -442,7 +442,7 @@ public class TestRegionReplicaReplicationEndpoint {
       HTU.getAdmin().deleteTable(toBeDisabledTable);
     } else if (disableReplication) {
       htd.setRegionReplication(regionReplication - 2);
-      HTU.getAdmin().modifyTable(toBeDisabledTable, htd);
+      HTU.getAdmin().modifyTable(htd);
       HTU.getAdmin().enableTable(toBeDisabledTable);
     }
     sinkWriter.append(toBeDisabledTable, encodedRegionName,
@@ -454,7 +454,7 @@ public class TestRegionReplicaReplicationEndpoint {
       // enable replication again so that we can verify replication
       HTU.getAdmin().disableTable(toBeDisabledTable); // disable the table
       htd.setRegionReplication(regionReplication);
-      HTU.getAdmin().modifyTable(toBeDisabledTable, htd);
+      HTU.getAdmin().modifyTable(htd);
       HTU.getAdmin().enableTable(toBeDisabledTable);
     }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
index cee34af..ad97790 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
@@ -1771,7 +1771,7 @@ public class TestAccessController extends SecureTestUtil {
 
       User newOwner = User.createUserForTesting(conf, "new_owner", new String[] {});
       htd.setOwner(newOwner);
-      admin.modifyTable(tableName, htd);
+      admin.modifyTable(htd);
 
       acl = systemUserConnection.getTable(AccessControlLists.ACL_TABLE_NAME);
       try {
@@ -2055,7 +2055,7 @@ public class TestAccessController extends SecureTestUtil {
   @Test
   public void testSnapshot() throws Exception {
     Admin admin = TEST_UTIL.getAdmin();
-    final HTableDescriptor htd = admin.getTableDescriptor(TEST_TABLE);
+    final HTableDescriptor htd = new HTableDescriptor(admin.getDescriptor(TEST_TABLE));
     final SnapshotDescription snapshot = new SnapshotDescription(
         TEST_TABLE.getNameAsString() + "-snapshot", TEST_TABLE);
     AccessTestAction snapshotAction = new AccessTestAction() {
@@ -2114,7 +2114,7 @@ public class TestAccessController extends SecureTestUtil {
   @Test
   public void testSnapshotWithOwner() throws Exception {
     Admin admin = TEST_UTIL.getAdmin();
-    final HTableDescriptor htd = admin.getTableDescriptor(TEST_TABLE);
+    final HTableDescriptor htd = new HTableDescriptor(admin.getDescriptor(TEST_TABLE));
     final SnapshotDescription snapshot = new SnapshotDescription(
         TEST_TABLE.getNameAsString() + "-snapshot", TEST_TABLE, null, USER_OWNER.getName());
 
@@ -2243,7 +2243,7 @@ public class TestAccessController extends SecureTestUtil {
         public Object run() throws Exception {
           try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());
               Admin admin = conn.getAdmin()) {
-            return Arrays.asList(admin.listTables());
+            return admin.listTableDescriptors();
           }
         }
       };
@@ -2253,7 +2253,7 @@ public class TestAccessController extends SecureTestUtil {
         public Object run() throws Exception {
           try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());
               Admin admin = conn.getAdmin()) {
-            return admin.getTableDescriptor(TEST_TABLE);
+            return admin.getDescriptor(TEST_TABLE);
           }
         }
       };
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCoprocessorWhitelistMasterObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCoprocessorWhitelistMasterObserver.java
index f9b610a..d7a0277 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCoprocessorWhitelistMasterObserver.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCoprocessorWhitelistMasterObserver.java
@@ -22,6 +22,7 @@ import static org.junit.Assert.fail;
 
 import java.io.IOException;
 import java.util.Optional;
+import java.util.regex.Pattern;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Coprocessor;
@@ -115,7 +116,7 @@ public class TestCoprocessorWhitelistMasterObserver extends SecureTestUtil {
       Coprocessor.PRIORITY_USER, null);
     LOG.info("Modifying Table");
     try {
-      connection.getAdmin().modifyTable(TEST_TABLE, htd);
+      connection.getAdmin().modifyTable(htd);
       fail("Expected coprocessor to raise IOException");
     } catch (IOException e) {
       // swallow exception from coprocessor
@@ -160,8 +161,8 @@ public class TestCoprocessorWhitelistMasterObserver extends SecureTestUtil {
       new Path(coprocessorPath),
       Coprocessor.PRIORITY_USER, null);
     LOG.info("Modifying Table");
-    admin.modifyTable(TEST_TABLE, htd);
-    assertEquals(1, t.getTableDescriptor().getCoprocessors().size());
+    admin.modifyTable(htd);
+    assertEquals(1, t.getDescriptor().getCoprocessorDescriptors().size());
     LOG.info("Done Modifying Table");
   }
 
@@ -288,8 +289,8 @@ public class TestCoprocessorWhitelistMasterObserver extends SecureTestUtil {
     }
     LOG.info("Done Creating Table");
     // ensure table was not created
-    assertEquals(new HTableDescriptor[0],
-      admin.listTables("^" + TEST_TABLE.getNameAsString() + "$"));
+    assertEquals(0,
+      admin.listTableDescriptors(Pattern.compile("^" + TEST_TABLE.getNameAsString() + "$")).size());
   }
 
   public static class TestRegionObserver implements RegionCoprocessor, RegionObserver {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestRpcAccessChecks.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestRpcAccessChecks.java
index 4fb5242..b95c776 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestRpcAccessChecks.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestRpcAccessChecks.java
@@ -218,7 +218,7 @@ public class TestRpcAccessChecks {
 
   @Test
   public void testEnableCatalogJanitor() throws Exception {
-    verifyAdminCheckForAction((admin) -> admin.enableCatalogJanitor(true));
+    verifyAdminCheckForAction((admin) -> admin.catalogJanitorSwitch(true));
   }
 
   @Test
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabels.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabels.java
index 770bf3e..4829d5c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabels.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabels.java
@@ -683,7 +683,7 @@ public abstract class TestVisibilityLabels {
       HTableDescriptor htd = new HTableDescriptor(LABELS_TABLE_NAME);
       htd.addFamily(new HColumnDescriptor("f1"));
       htd.addFamily(new HColumnDescriptor("f2"));
-      admin.modifyTable(LABELS_TABLE_NAME, htd);
+      admin.modifyTable(htd);
       fail("Lables table should not get altered by user.");
     } catch (Exception e) {
     }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java
index 75dfc30..d39c0e6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java
@@ -777,7 +777,7 @@ public final class SnapshotTestingUtils {
     byte[][] splitKeys = getSplitKeys(nRegions);
     util.createTable(builder.build(), splitKeys);
     assertEquals((splitKeys.length + 1) * regionReplication,
-        util.getAdmin().getTableRegions(tableName).size());
+        util.getAdmin().getRegions(tableName).size());
   }
 
   public static byte[][] getSplitKeys() {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java
index 5a589d4..2f84c95 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java
@@ -34,10 +34,10 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.SnapshotDescription;
 import org.apache.hadoop.hbase.client.SnapshotType;
 import org.apache.hadoop.hbase.client.Table;
@@ -254,13 +254,13 @@ public class TestFlushSnapshotFromClient {
     // make sure the table doesn't exist
     boolean fail = false;
     do {
-    try {
-      admin.getTableDescriptor(tableName);
-      fail = true;
-      LOG.error("Table:" + tableName + " already exists, checking a new name");
-      tableName = TableName.valueOf(tableName+"!");
-    } catch (TableNotFoundException e) {
-      fail = false;
+      try {
+        admin.getDescriptor(tableName);
+        fail = true;
+        LOG.error("Table:" + tableName + " already exists, checking a new name");
+        tableName = TableName.valueOf(tableName + "!");
+      } catch (TableNotFoundException e) {
+        fail = false;
       }
     } while (fail);
 
@@ -280,7 +280,7 @@ public class TestFlushSnapshotFromClient {
         .setType(SnapshotProtos.SnapshotDescription.Type.FLUSH).build();
 
     // take the snapshot async
-    admin.takeSnapshotAsync(
+    admin.snapshotAsync(
       new SnapshotDescription("asyncSnapshot", TABLE_NAME, SnapshotType.FLUSH));
 
     // constantly loop, looking for the snapshot to complete
@@ -311,15 +311,15 @@ public class TestFlushSnapshotFromClient {
     SnapshotTestingUtils.waitForTableToBeOnline(UTIL, cloneBeforeMergeName);
 
     // Merge two regions
-    List<HRegionInfo> regions = admin.getTableRegions(TABLE_NAME);
-    Collections.sort(regions, new Comparator<HRegionInfo>() {
+    List<RegionInfo> regions = admin.getRegions(TABLE_NAME);
+    Collections.sort(regions, new Comparator<RegionInfo>() {
       @Override
-      public int compare(HRegionInfo r1, HRegionInfo r2) {
+      public int compare(RegionInfo r1, RegionInfo r2) {
         return Bytes.compareTo(r1.getStartKey(), r2.getStartKey());
       }
     });
 
-    int numRegions = admin.getTableRegions(TABLE_NAME).size();
+    int numRegions = admin.getRegions(TABLE_NAME).size();
     int numRegionsAfterMerge = numRegions - 2;
     admin.mergeRegionsAsync(regions.get(1).getEncodedNameAsBytes(),
         regions.get(2).getEncodedNameAsBytes(), true);
@@ -328,7 +328,7 @@ public class TestFlushSnapshotFromClient {
 
     // Verify that there's one region less
     waitRegionsAfterMerge(numRegionsAfterMerge);
-    assertEquals(numRegionsAfterMerge, admin.getTableRegions(TABLE_NAME).size());
+    assertEquals(numRegionsAfterMerge, admin.getRegions(TABLE_NAME).size());
 
     // Clone the table
     TableName cloneAfterMergeName = TableName.valueOf("cloneAfterMerge");
@@ -353,15 +353,15 @@ public class TestFlushSnapshotFromClient {
     SnapshotTestingUtils.loadData(UTIL, TABLE_NAME, numRows, TEST_FAM);
 
     // Merge two regions
-    List<HRegionInfo> regions = admin.getTableRegions(TABLE_NAME);
-    Collections.sort(regions, new Comparator<HRegionInfo>() {
+    List<RegionInfo> regions = admin.getRegions(TABLE_NAME);
+    Collections.sort(regions, new Comparator<RegionInfo>() {
       @Override
-      public int compare(HRegionInfo r1, HRegionInfo r2) {
+      public int compare(RegionInfo r1, RegionInfo r2) {
         return Bytes.compareTo(r1.getStartKey(), r2.getStartKey());
       }
     });
 
-    int numRegions = admin.getTableRegions(TABLE_NAME).size();
+    int numRegions = admin.getRegions(TABLE_NAME).size();
     int numRegionsAfterMerge = numRegions - 2;
     admin.mergeRegionsAsync(regions.get(1).getEncodedNameAsBytes(),
         regions.get(2).getEncodedNameAsBytes(), true);
@@ -369,7 +369,7 @@ public class TestFlushSnapshotFromClient {
         regions.get(5).getEncodedNameAsBytes(), true);
 
     waitRegionsAfterMerge(numRegionsAfterMerge);
-    assertEquals(numRegionsAfterMerge, admin.getTableRegions(TABLE_NAME).size());
+    assertEquals(numRegionsAfterMerge, admin.getRegions(TABLE_NAME).size());
 
     // Take a snapshot
     String snapshotName = "snapshotAfterMerge";
@@ -436,7 +436,7 @@ public class TestFlushSnapshotFromClient {
         try {
           LOG.info("Submitting snapshot request: " + ClientSnapshotDescriptionUtils
               .toString(ProtobufUtil.createHBaseProtosSnapshotDesc(ss)));
-          admin.takeSnapshotAsync(ss);
+          admin.snapshotAsync(ss);
         } catch (Exception e) {
           LOG.info("Exception during snapshot request: " + ClientSnapshotDescriptionUtils.toString(
             ProtobufUtil.createHBaseProtosSnapshotDesc(ss))
@@ -514,7 +514,7 @@ public class TestFlushSnapshotFromClient {
       throws IOException, InterruptedException {
     // Verify that there's one region less
     long startTime = System.currentTimeMillis();
-    while (admin.getTableRegions(TABLE_NAME).size() != numRegionsAfterMerge) {
+    while (admin.getRegions(TABLE_NAME).size() != numRegionsAfterMerge) {
       // This may be flaky... if after 15sec the merge is not complete give up
       // it will fail in the assertEquals(numRegionsAfterMerge).
       if ((System.currentTimeMillis() - startTime) > 15000)
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreFlushSnapshotFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreFlushSnapshotFromClient.java
index cfe3b15..5ea8f3f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreFlushSnapshotFromClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreFlushSnapshotFromClient.java
@@ -61,9 +61,9 @@ public class TestRestoreFlushSnapshotFromClient {
 
   protected final byte[] FAMILY = Bytes.toBytes("cf");
 
-  protected byte[] snapshotName0;
-  protected byte[] snapshotName1;
-  protected byte[] snapshotName2;
+  protected String snapshotName0;
+  protected String snapshotName1;
+  protected String snapshotName2;
   protected int snapshot0Rows;
   protected int snapshot1Rows;
   protected TableName tableName;
@@ -108,9 +108,9 @@ public class TestRestoreFlushSnapshotFromClient {
 
     long tid = System.currentTimeMillis();
     tableName = TableName.valueOf("testtb-" + tid);
-    snapshotName0 = Bytes.toBytes("snaptb0-" + tid);
-    snapshotName1 = Bytes.toBytes("snaptb1-" + tid);
-    snapshotName2 = Bytes.toBytes("snaptb2-" + tid);
+    snapshotName0 = "snaptb0-" + tid;
+    snapshotName1 = "snaptb1-" + tid;
+    snapshotName2 = "snaptb2-" + tid;
 
     // create Table and disable it
     createTable();
@@ -121,7 +121,7 @@ public class TestRestoreFlushSnapshotFromClient {
     logFSTree();
 
     // take a snapshot
-    admin.snapshot(Bytes.toString(snapshotName0), tableName, SnapshotType.FLUSH);
+    admin.snapshot(snapshotName0, tableName, SnapshotType.FLUSH);
 
     LOG.info("=== after snapshot with 500 rows");
     logFSTree();
@@ -133,7 +133,7 @@ public class TestRestoreFlushSnapshotFromClient {
     logFSTree();
 
     // take a snapshot of the updated table
-    admin.snapshot(Bytes.toString(snapshotName1), tableName, SnapshotType.FLUSH);
+    admin.snapshot(snapshotName1, tableName, SnapshotType.FLUSH);
     LOG.info("=== after snapshot with 1000 rows");
     logFSTree();
     table.close();
@@ -184,7 +184,7 @@ public class TestRestoreFlushSnapshotFromClient {
     testCloneSnapshot(clonedTableName, snapshotName1, snapshot1Rows);
   }
 
-  private void testCloneSnapshot(final TableName tableName, final byte[] snapshotName,
+  private void testCloneSnapshot(final TableName tableName, final String snapshotName,
       int snapshotRows) throws IOException, InterruptedException {
     // create a new table from snapshot
     admin.cloneSnapshot(snapshotName, tableName);
@@ -198,7 +198,7 @@ public class TestRestoreFlushSnapshotFromClient {
     TableName clonedTableName = TableName.valueOf("clonedtb-" + System.currentTimeMillis());
     admin.cloneSnapshot(snapshotName0, clonedTableName);
     verifyRowCount(UTIL, clonedTableName, snapshot0Rows);
-    admin.snapshot(Bytes.toString(snapshotName2), clonedTableName, SnapshotType.FLUSH);
+    admin.snapshot(snapshotName2, clonedTableName, SnapshotType.FLUSH);
     UTIL.deleteTable(clonedTableName);
 
     admin.cloneSnapshot(snapshotName2, clonedTableName);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckMOB.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckMOB.java
index 0855559..3686150 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckMOB.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckMOB.java
@@ -77,7 +77,7 @@ public class TestHBaseFsckMOB extends BaseTestHBaseFsck {
     connection = (ClusterConnection) TEST_UTIL.getConnection();
 
     admin = connection.getAdmin();
-    admin.setBalancerRunning(false, true);
+    admin.balancerSwitch(false, true);
 
     TEST_UTIL.waitUntilAllRegionsAssigned(TableName.META_TABLE_NAME);
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMiniClusterLoadSequential.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMiniClusterLoadSequential.java
index f2973d6..5af2372 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMiniClusterLoadSequential.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMiniClusterLoadSequential.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
@@ -182,9 +183,9 @@ public class TestMiniClusterLoadSequential {
     return 1000;
   }
 
-  protected HColumnDescriptor getColumnDesc(Admin admin)
+  protected ColumnFamilyDescriptor getColumnDesc(Admin admin)
       throws TableNotFoundException, IOException {
-    return admin.getTableDescriptor(TABLE).getFamily(CF);
+    return admin.getDescriptor(TABLE).getColumnFamily(CF);
   }
 
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java
index 531d08c..1329f44 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java
@@ -22,7 +22,6 @@ import static org.junit.Assert.assertEquals;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -47,6 +46,7 @@ import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
@@ -97,7 +97,7 @@ public class OfflineMetaRebuildTestCore {
     TEST_UTIL.startMiniCluster(3);
     conf = TEST_UTIL.getConfiguration();
     this.connection = ConnectionFactory.createConnection(conf);
-    assertEquals(0, TEST_UTIL.getAdmin().listTables().length);
+    assertEquals(0, TEST_UTIL.getAdmin().listTableDescriptors().size());
 
     // setup the table
     table = TableName.valueOf(TABLE_BASE + "-" + tableIdx);
@@ -109,7 +109,7 @@ public class OfflineMetaRebuildTestCore {
         + " entries.");
     assertEquals(16, tableRowCount(conf, table));
     TEST_UTIL.getAdmin().disableTable(table);
-    assertEquals(1, TEST_UTIL.getAdmin().listTables().length);
+    assertEquals(1, TEST_UTIL.getAdmin().listTableDescriptors().size());
   }
 
   @After
@@ -280,13 +280,11 @@ public class OfflineMetaRebuildTestCore {
     return MetaTableAccessor.fullScanRegions(TEST_UTIL.getConnection()).size();
   }
 
-  protected HTableDescriptor[] getTables(final Configuration configuration) throws IOException {
-    HTableDescriptor[] htbls = null;
+  protected List<TableDescriptor> getTables(final Configuration configuration) throws IOException {
     try (Connection connection = ConnectionFactory.createConnection(configuration)) {
       try (Admin admin = connection.getAdmin()) {
-        htbls = admin.listTables();
+        return admin.listTableDescriptors();
       }
     }
-    return htbls;
   }
 }
diff --git a/hbase-shell/src/main/ruby/hbase/admin.rb b/hbase-shell/src/main/ruby/hbase/admin.rb
index 6b0e7c0..31545b5 100644
--- a/hbase-shell/src/main/ruby/hbase/admin.rb
+++ b/hbase-shell/src/main/ruby/hbase/admin.rb
@@ -144,8 +144,8 @@ module Hbase
       split_point_bytes = nil
       split_point_bytes = split_point.to_java_bytes unless split_point.nil?
       begin
-        @admin.splitRegion(table_or_region_name.to_java_bytes, split_point_bytes)
-      rescue java.lang.IllegalArgumentException => e
+        @admin.splitRegionAsync(table_or_region_name.to_java_bytes, split_point_bytes).get
+      rescue java.lang.IllegalArgumentException, org.apache.hadoop.hbase.UnknownRegionException => e
         @admin.split(TableName.valueOf(table_or_region_name), split_point_bytes)
       end
     end
@@ -154,33 +154,26 @@ module Hbase
     # Enable/disable one split or merge switch
     # Returns previous switch setting.
     def splitormerge_switch(type, enabled)
-      switch_type = nil
       if type == 'SPLIT'
-        switch_type = org.apache.hadoop.hbase.client::MasterSwitchType::SPLIT
+        @admin.splitSwitch(java.lang.Boolean.valueOf(enabled), java.lang.Boolean.valueOf(false))
       elsif type == 'MERGE'
-        switch_type = org.apache.hadoop.hbase.client::MasterSwitchType::MERGE
+        @admin.mergeSwitch(java.lang.Boolean.valueOf(enabled), java.lang.Boolean.valueOf(false))
       else
         raise ArgumentError, 'only SPLIT or MERGE accepted for type!'
       end
-      @admin.setSplitOrMergeEnabled(
-        java.lang.Boolean.valueOf(enabled), java.lang.Boolean.valueOf(false),
-        switch_type
-      )[0]
     end
 
     #----------------------------------------------------------------------------------------------
     # Query the current state of the split or merge switch.
     # Returns the switch's state (true is enabled).
     def splitormerge_enabled(type)
-      switch_type = nil
       if type == 'SPLIT'
-        switch_type = org.apache.hadoop.hbase.client::MasterSwitchType::SPLIT
+        @admin.isSplitEnabled
       elsif type == 'MERGE'
-        switch_type = org.apache.hadoop.hbase.client::MasterSwitchType::MERGE
+        @admin.isMergeEnabled
       else
         raise ArgumentError, 'only SPLIT or MERGE accepted for type!'
       end
-      @admin.isSplitOrMergeEnabled(switch_type)
     end
 
     def locate_region(table_name, row_key)
@@ -203,7 +196,7 @@ module Hbase
     # Enable/disable balancer
     # Returns previous balancer switch setting.
     def balance_switch(enableDisable)
-      @admin.setBalancerRunning(
+      @admin.balancerSwitch(
         java.lang.Boolean.valueOf(enableDisable), java.lang.Boolean.valueOf(false)
       )
     end
@@ -232,7 +225,7 @@ module Hbase
     # Enable/disable region normalizer
     # Returns previous normalizer switch setting.
     def normalizer_switch(enableDisable)
-      @admin.setNormalizerRunning(java.lang.Boolean.valueOf(enableDisable))
+      @admin.normalizerSwitch(java.lang.Boolean.valueOf(enableDisable))
     end
 
     #----------------------------------------------------------------------------------------------
@@ -253,14 +246,14 @@ module Hbase
     # Request a scan of the catalog table (for garbage collection)
     # Returns an int signifying the number of entries cleaned
     def catalogjanitor_run
-      @admin.runCatalogScan
+      @admin.runCatalogJanitor
     end
 
     #----------------------------------------------------------------------------------------------
     # Enable/disable the catalog janitor
     # Returns previous catalog janitor switch setting.
     def catalogjanitor_switch(enableDisable)
-      @admin.enableCatalogJanitor(java.lang.Boolean.valueOf(enableDisable))
+      @admin.catalogJanitorSwitch(java.lang.Boolean.valueOf(enableDisable))
     end
 
     #----------------------------------------------------------------------------------------------
@@ -280,7 +273,7 @@ module Hbase
     # Enable/disable the cleaner chore
     # Returns previous cleaner switch setting.
     def cleaner_chore_switch(enableDisable)
-      @admin.setCleanerChoreRunning(java.lang.Boolean.valueOf(enableDisable))
+      @admin.cleanerChoreSwitch(java.lang.Boolean.valueOf(enableDisable))
     end
 
     #----------------------------------------------------------------------------------------------
@@ -301,8 +294,17 @@ module Hbase
     #----------------------------------------------------------------------------------------------
     # Enables all tables matching the given regex
     def enable_all(regex)
-      regex = regex.to_s
-      @admin.enableTables(Pattern.compile(regex))
+      pattern = Pattern.compile(regex.to_s)
+      failed = java.util.ArrayList.new
+      admin.listTableNames(pattern).each do |table_name|
+        begin
+          admin.enableTable(table_name)
+        rescue java.io.IOException => e
+          puts "table:#{table_name}, error:#{e.toString}"
+          failed.add(table_name)
+        end
+      end
+      @failed
     end
 
     #----------------------------------------------------------------------------------------------
@@ -317,7 +319,16 @@ module Hbase
     # Disables all tables matching the given regex
     def disable_all(regex)
       pattern = Pattern.compile(regex.to_s)
-      @admin.disableTables(pattern).map { |t| t.getTableName.getNameAsString }
+      failed = java.util.ArrayList.new
+      admin.listTableNames(pattern).each do |table_name|
+        begin
+          admin.disableTable(table_name)
+        rescue java.io.IOException => e
+          puts "table:#{table_name}, error:#{e.toString}"
+          failed.add(table_name)
+        end
+      end
+      @failed
     end
 
     #---------------------------------------------------------------------------------------------
@@ -347,8 +358,16 @@ module Hbase
     # Drops a table
     def drop_all(regex)
       pattern = Pattern.compile(regex.to_s)
-      failed = @admin.deleteTables(pattern).map { |t| t.getTableName.getNameAsString }
-      failed
+      failed = java.util.ArrayList.new
+      admin.listTableNames(pattern).each do |table_name|
+        begin
+          admin.deleteTable(table_name)
+        rescue java.io.IOException => e
+          puts puts "table:#{table_name}, error:#{e.toString}"
+          failed.add(table_name)
+        end
+      end
+      @failed
     end
 
     #----------------------------------------------------------------------------------------------
@@ -492,17 +511,17 @@ module Hbase
     # Returns table's structure description
     def describe(table_name)
       tableExists(table_name)
-      @admin.getTableDescriptor(TableName.valueOf(table_name)).to_s
+      @admin.getDescriptor(TableName.valueOf(table_name)).to_s
     end
 
     def get_column_families(table_name)
       tableExists(table_name)
-      @admin.getTableDescriptor(TableName.valueOf(table_name)).getColumnFamilies
+      @admin.getDescriptor(TableName.valueOf(table_name)).getColumnFamilies
     end
 
     def get_table_attributes(table_name)
       tableExists(table_name)
-      @admin.getTableDescriptor(TableName.valueOf(table_name)).toStringTableAttributes
+      @admin.getDescriptor(TableName.valueOf(table_name)).toStringTableAttributes
     end
 
     #----------------------------------------------------------------------------------------------
@@ -510,7 +529,7 @@ module Hbase
     def truncate(table_name_str)
       puts "Truncating '#{table_name_str}' table (it may take a while):"
       table_name = TableName.valueOf(table_name_str)
-      table_description = @admin.getTableDescriptor(table_name)
+      table_description = @admin.getDescriptor(table_name)
       raise ArgumentError, "Table #{table_name_str} is not enabled. Enable it first." unless
           enabled?(table_name_str)
       puts 'Disabling table...'
@@ -551,7 +570,7 @@ module Hbase
         locator.close
       end
 
-      table_description = @admin.getTableDescriptor(table_name)
+      table_description = @admin.getDescriptor(table_name)
       puts 'Disabling table...'
       disable(table_name_str)
 
@@ -627,7 +646,7 @@ module Hbase
       table_name = TableName.valueOf(table_name_str)
 
       # Get table descriptor
-      htd = org.apache.hadoop.hbase.HTableDescriptor.new(@admin.getTableDescriptor(table_name))
+      htd = org.apache.hadoop.hbase.HTableDescriptor.new(@admin.getDescriptor(table_name))
       hasTableUpdate = false
 
       # Process all args
@@ -742,17 +761,17 @@ module Hbase
 
       # Bulk apply all table modifications.
       if hasTableUpdate
-        @admin.modifyTable(table_name, htd)
+        future = @admin.modifyTableAsync(htd)
 
         if wait == true
           puts 'Updating all regions with the new schema...'
-          alter_status(table_name_str)
+          future.get
         end
       end
     end
 
     def status(format, type)
-      status = @admin.getClusterStatus
+      status = org.apache.hadoop.hbase.ClusterStatus.new(@admin.getClusterMetrics)
       if format == 'detailed'
         puts(format('version %s', status.getHBaseVersion))
         # Put regions in transition first because usually empty
@@ -767,7 +786,7 @@ module Hbase
           puts(format('    %s:%d %d', server.getHostname, server.getPort, server.getStartcode))
         end
 
-        master_coprocs = java.util.Arrays.toString(@admin.getMasterCoprocessors)
+        master_coprocs = @admin.getMasterCoprocessorNames.toString
         unless master_coprocs.nil?
           puts(format('master coprocessors: %s', master_coprocs))
         end
@@ -1124,13 +1143,13 @@ module Hbase
     #----------------------------------------------------------------------------------------------
     # Returns the ClusterStatus of the cluster
     def getClusterStatus
-      @admin.getClusterStatus
+      org.apache.hadoop.hbase.ClusterStatus.new(@admin.getClusterMetrics)
     end
 
     #----------------------------------------------------------------------------------------------
     # Returns a list of regionservers
     def getRegionServers
-      @admin.getClusterStatus.getServers.map { |serverName| serverName }
+      org.apache.hadoop.hbase.ClusterStatus.new(@admin.getClusterMetrics).getServers.map { |serverName| serverName }
     end
 
     #----------------------------------------------------------------------------------------------
@@ -1393,7 +1412,7 @@ module Hbase
     #----------------------------------------------------------------------------------------------
     # List live region servers
     def list_liveservers
-      @admin.getClusterStatus.getServers.to_a
+      org.apache.hadoop.hbase.ClusterStatus.new(@admin.getClusterMetrics).getServers.to_a
     end
 
     #---------------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/quotas.rb b/hbase-shell/src/main/ruby/hbase/quotas.rb
index f67ee75..d4d73e9 100644
--- a/hbase-shell/src/main/ruby/hbase/quotas.rb
+++ b/hbase-shell/src/main/ruby/hbase/quotas.rb
@@ -238,30 +238,25 @@ module Hbase
       raise(ArgumentError, 'Unexpected arguments: ' + args.inspect) unless args.empty?
 
       # Start the scanner
-      scanner = @admin.getQuotaRetriever(filter)
-      begin
-        iter = scanner.iterator
+      quotas = @admin.getQuota(filter)
+      iter = quotas.iterator
 
-        # Iterate results
-        while iter.hasNext
-          break if limit > 0 && count >= limit
+      # Iterate results
+      while iter.hasNext
+        break if limit > 0 && count >= limit
 
-          settings = iter.next
-          owner = {
-            USER => settings.getUserName,
-            TABLE => settings.getTableName,
-            NAMESPACE => settings.getNamespace,
-            REGIONSERVER => settings.getRegionServer
-          }.delete_if { |_k, v| v.nil? }.map { |k, v| k.to_s + ' => ' + v.to_s } * ', '
+        settings = iter.next
+        owner = {
+          USER => settings.getUserName,
+          TABLE => settings.getTableName,
+          NAMESPACE => settings.getNamespace,
+          REGIONSERVER => settings.getRegionServer
+        }.delete_if { |_k, v| v.nil? }.map { |k, v| k.to_s + ' => ' + v.to_s } * ', '
 
-          yield owner, settings.to_s
+        yield owner, settings.to_s
 
-          count += 1
-        end
-      ensure
-        scanner.close
+        count += 1
       end
-
       count
     end
 
diff --git a/hbase-shell/src/main/ruby/hbase/security.rb b/hbase-shell/src/main/ruby/hbase/security.rb
index 5275898..4639c71 100644
--- a/hbase-shell/src/main/ruby/hbase/security.rb
+++ b/hbase-shell/src/main/ruby/hbase/security.rb
@@ -66,7 +66,7 @@ module Hbase
             raise(ArgumentError, "Can't find a table: #{table_name}") unless exists?(table_name)
 
             tableName = org.apache.hadoop.hbase.TableName.valueOf(table_name)
-            htd = @admin.getTableDescriptor(tableName)
+            htd = org.apache.hadoop.hbase.HTableDescriptor.new(@admin.getDescriptor(tableName))
 
             unless family.nil?
               raise(ArgumentError, "Can't find a family: #{family}") unless htd.hasFamily(family.to_java_bytes)
@@ -111,7 +111,7 @@ module Hbase
             raise(ArgumentError, "Can't find a table: #{table_name}") unless exists?(table_name)
 
             tableName = org.apache.hadoop.hbase.TableName.valueOf(table_name)
-            htd = @admin.getTableDescriptor(tableName)
+            htd = org.apache.hadoop.hbase.HTableDescriptor.new(@admin.getDescriptor(tableName))
 
             unless family.nil?
               raise(ArgumentError, "Can't find a family: #{family}") unless htd.hasFamily(family.to_java_bytes)
diff --git a/hbase-shell/src/main/ruby/hbase/taskmonitor.rb b/hbase-shell/src/main/ruby/hbase/taskmonitor.rb
index 76f8711..ca4180a 100644
--- a/hbase-shell/src/main/ruby/hbase/taskmonitor.rb
+++ b/hbase-shell/src/main/ruby/hbase/taskmonitor.rb
@@ -79,7 +79,7 @@ module Hbase
       java_import 'java.io.InputStreamReader'
       java_import 'org.apache.hbase.thirdparty.com.google.gson.JsonParser'
 
-      infoport = @admin.getClusterStatus.getLoad(host).getInfoServerPort.to_s
+      infoport = org.apache.hadoop.hbase.ClusterStatus.new(@admin.getClusterMetrics).getLoad(host).getInfoServerPort.to_s
 
       # Note: This condition use constants from hbase-server
       # if (!@admin.getConfiguration().getBoolean(org.apache.hadoop.hbase.http.ServerConfigurationKeys::HBASE_SSL_ENABLED_KEY,
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftHBaseServiceHandler.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftHBaseServiceHandler.java
index a112cef..53fbfff 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftHBaseServiceHandler.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftHBaseServiceHandler.java
@@ -54,7 +54,7 @@ import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.regex.Pattern;
-
+import org.apache.commons.lang3.NotImplementedException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HRegionLocation;
@@ -705,13 +705,7 @@ public class ThriftHBaseServiceHandler extends HBaseServiceHandler implements TH
   @Override
   public boolean isTableAvailableWithSplit(TTableName tableName, List<ByteBuffer> splitKeys)
       throws TIOError, TException {
-    try {
-      TableName table = tableNameFromThrift(tableName);
-      byte[][] split = splitKeyFromThrift(splitKeys);
-      return connectionCache.getAdmin().isTableAvailable(table, split);
-    } catch (IOException e) {
-      throw getTIOError(e);
-    }
+    throw new NotImplementedException("isTableAvailableWithSplit not supported");
   }
 
   @Override
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java
index 888256e..aa0c2a8 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java
@@ -31,8 +31,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.CacheEvictionStats;
 import org.apache.hadoop.hbase.ClusterMetrics;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.NamespaceNotFoundException;
 import org.apache.hadoop.hbase.RegionMetrics;
@@ -53,7 +51,6 @@ import org.apache.hadoop.hbase.client.replication.TableCFs;
 import org.apache.hadoop.hbase.client.security.SecurityCapability;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
 import org.apache.hadoop.hbase.quotas.QuotaFilter;
-import org.apache.hadoop.hbase.quotas.QuotaRetriever;
 import org.apache.hadoop.hbase.quotas.QuotaSettings;
 import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
@@ -68,7 +65,6 @@ import org.apache.hadoop.hbase.thrift2.generated.TNamespaceDescriptor;
 import org.apache.hadoop.hbase.thrift2.generated.TTableDescriptor;
 import org.apache.hadoop.hbase.thrift2.generated.TTableName;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.Pair;
 import org.apache.thrift.TException;
 import org.apache.thrift.transport.TTransport;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -112,7 +108,7 @@ public class ThriftAdmin implements Admin {
   }
 
   @Override
-  public void close() throws IOException {
+  public void close() {
     transport.close();
   }
 
@@ -137,40 +133,16 @@ public class ThriftAdmin implements Admin {
   }
 
   @Override
-  public HTableDescriptor[] listTables() throws IOException {
-    return listTables((String)null);
-  }
-
-  @Override
   public List<TableDescriptor> listTableDescriptors() throws IOException {
     return listTableDescriptors((Pattern) null);
   }
 
   @Override
-  public HTableDescriptor[] listTables(Pattern pattern) throws IOException {
-    String regex = (pattern == null ? null : pattern.toString());
-    return listTables(regex);
-  }
-
-  @Override
   public List<TableDescriptor> listTableDescriptors(Pattern pattern) throws IOException {
     return listTableDescriptors(pattern, false);
   }
 
   @Override
-  public HTableDescriptor[] listTables(String regex) throws IOException {
-    return listTables(regex, false);
-  }
-
-  @Override
-  public HTableDescriptor[] listTables(Pattern pattern, boolean includeSysTables)
-      throws IOException {
-    String regex = (pattern == null ? null : pattern.toString());
-    return listTables(regex, includeSysTables);
-
-  }
-
-  @Override
   public List<TableDescriptor> listTableDescriptors(Pattern pattern, boolean includeSysTables)
       throws IOException {
     try {
@@ -185,20 +157,8 @@ public class ThriftAdmin implements Admin {
   }
 
   @Override
-  public HTableDescriptor[] listTables(String regex, boolean includeSysTables) throws IOException {
-    try {
-      List<TTableDescriptor> tTableDescriptors = client
-          .getTableDescriptorsByPattern(regex, includeSysTables);
-      return ThriftUtilities.hTableDescriptorsFromThrift(tTableDescriptors);
-
-    } catch (TException e) {
-      throw new IOException(e);
-    }
-  }
-
-  @Override
   public TableName[] listTableNames() throws IOException {
-    return listTableNames((String)null);
+    return listTableNames(null);
   }
 
   @Override
@@ -207,18 +167,8 @@ public class ThriftAdmin implements Admin {
   }
 
   @Override
-  public TableName[] listTableNames(String regex) throws IOException {
-    return listTableNames(regex, false);
-  }
-
-  @Override
   public TableName[] listTableNames(Pattern pattern, boolean includeSysTables) throws IOException {
     String regex = (pattern == null ? null : pattern.toString());
-    return listTableNames(regex, includeSysTables);
-  }
-
-  @Override
-  public TableName[] listTableNames(String regex, boolean includeSysTables) throws IOException {
     try {
       List<TTableName> tTableNames = client.getTableNamesByPattern(regex, includeSysTables);
       return ThriftUtilities.tableNamesArrayFromThrift(tTableNames);
@@ -228,18 +178,6 @@ public class ThriftAdmin implements Admin {
   }
 
   @Override
-  public HTableDescriptor getTableDescriptor(TableName tableName)
-      throws TableNotFoundException, IOException {
-    TTableName tTableName = ThriftUtilities.tableNameFromHBase(tableName);
-    try {
-      TTableDescriptor tTableDescriptor = client.getTableDescriptor(tTableName);
-      return ThriftUtilities.hTableDescriptorFromThrift(tTableDescriptor);
-    } catch (TException e) {
-      throw new IOException(e);
-    }
-  }
-
-  @Override
   public TableDescriptor getDescriptor(TableName tableName)
       throws TableNotFoundException, IOException {
     TTableName tTableName = ThriftUtilities.tableNameFromHBase(tableName);
@@ -252,16 +190,6 @@ public class ThriftAdmin implements Admin {
   }
 
   @Override
-  public HTableDescriptor[] listTableDescriptorsByNamespace(String name) throws IOException {
-    try {
-      List<TTableDescriptor> tTableDescriptors = client.getTableDescriptorsByNamespace(name);
-      return ThriftUtilities.hTableDescriptorsFromThrift(tTableDescriptors);
-    } catch (TException e) {
-      throw new IOException(e);
-    }
-  }
-
-  @Override
   public List<TableDescriptor> listTableDescriptorsByNamespace(byte[] name) throws IOException {
     try {
       List<TTableDescriptor> tTableDescriptors = client
@@ -388,17 +316,6 @@ public class ThriftAdmin implements Admin {
   }
 
   @Override
-  public boolean isTableAvailable(TableName tableName, byte[][] splitKeys) throws IOException {
-    TTableName tTableName = ThriftUtilities.tableNameFromHBase(tableName);
-    List<ByteBuffer> splitKeyInBuffer = ThriftUtilities.splitKeyFromHBase(splitKeys);
-    try {
-      return client.isTableAvailableWithSplit(tTableName, splitKeyInBuffer);
-    } catch (TException e) {
-      throw new IOException(e);
-    }
-  }
-
-  @Override
   public void addColumnFamily(TableName tableName, ColumnFamilyDescriptor columnFamily)
       throws IOException {
     TTableName tTableName = ThriftUtilities.tableNameFromHBase(tableName);
@@ -412,11 +329,6 @@ public class ThriftAdmin implements Admin {
   }
 
   @Override
-  public void deleteColumn(TableName tableName, byte[] columnFamily) throws IOException {
-    deleteColumnFamily(tableName, columnFamily);
-  }
-
-  @Override
   public void deleteColumnFamily(TableName tableName, byte[] columnFamily) throws IOException {
     TTableName tTableName = ThriftUtilities.tableNameFromHBase(tableName);
     try {
@@ -440,11 +352,6 @@ public class ThriftAdmin implements Admin {
   }
 
   @Override
-  public void modifyTable(TableName tableName, TableDescriptor td) throws IOException {
-    modifyTable(td);
-  }
-
-  @Override
   public void modifyTable(TableDescriptor td) throws IOException {
     TTableDescriptor tTableDescriptor = ThriftUtilities
         .tableDescriptorFromHBase(td);
@@ -525,85 +432,12 @@ public class ThriftAdmin implements Admin {
   }
 
   @Override
-  public HTableDescriptor[] disableTables(String regex) throws IOException {
-    throw new NotImplementedException("disableTables by pattern not supported in ThriftAdmin");
-  }
-
-  @Override
-  public HTableDescriptor[] disableTables(Pattern pattern) throws IOException {
-    throw new NotImplementedException("disableTables by pattern not supported in ThriftAdmin");
-  }
-
-  @Override
-  public HTableDescriptor[] enableTables(String regex) throws IOException {
-    throw new NotImplementedException("enableTables by pattern not supported in ThriftAdmin");
-  }
-
-  @Override
-  public HTableDescriptor[] enableTables(Pattern pattern) throws IOException {
-    throw new NotImplementedException("enableTables by pattern not supported in ThriftAdmin");
-  }
-
-  @Override
-  public HTableDescriptor[] deleteTables(String regex) throws IOException {
-    throw new NotImplementedException("deleteTables by pattern not supported in ThriftAdmin");
-  }
-
-  @Override
-  public HTableDescriptor[] deleteTables(Pattern pattern) throws IOException {
-    throw new NotImplementedException("deleteTables by pattern not supported in ThriftAdmin");
-
-  }
-
-  @Override
-  public HTableDescriptor[] getTableDescriptorsByTableName(List<TableName> tableNames)
-      throws IOException {
-    throw new NotImplementedException("getTableDescriptorsByTableName not supported in ThriftAdmin"
-        + ", use getDescriptor to get descriptors one by one");
-  }
-
-  @Override
   public List<TableDescriptor> listTableDescriptors(List<TableName> tableNames) throws IOException {
     throw new NotImplementedException("listTableDescriptors not supported in ThriftAdmin"
         + ", use getDescriptor to get descriptors one by one");
   }
 
   @Override
-  public HTableDescriptor[] getTableDescriptors(List<String> names) throws IOException {
-    throw new NotImplementedException("getTableDescriptors not supported in ThriftAdmin"
-        + ", use getDescriptor to get descriptors one by one");
-  }
-
-  @Override
-  public void closeRegion(String regionname, String serverName) {
-    throw new NotImplementedException("closeRegion not supported in ThriftAdmin");
-
-  }
-
-  @Override
-  public void closeRegion(byte[] regionname, String serverName) {
-    throw new NotImplementedException("closeRegion not supported in ThriftAdmin");
-
-  }
-
-  @Override
-  public boolean closeRegionWithEncodedRegionName(String encodedRegionName, String serverName) {
-    throw new NotImplementedException(
-        "closeRegionWithEncodedRegionName not supported in ThriftAdmin");
-  }
-
-  @Override
-  public void closeRegion(ServerName sn, HRegionInfo hri) {
-    throw new NotImplementedException("closeRegion not supported in ThriftAdmin");
-
-  }
-
-  @Override
-  public List<HRegionInfo> getOnlineRegions(ServerName sn) {
-    throw new NotImplementedException("getOnlineRegions not supported in ThriftAdmin");
-  }
-
-  @Override
   public List<RegionInfo> getRegions(ServerName serverName) {
     throw new NotImplementedException("getRegions not supported in ThriftAdmin");
   }
@@ -811,12 +645,6 @@ public class ThriftAdmin implements Admin {
   }
 
   @Override
-  public void mergeRegions(byte[] nameOfRegionA, byte[] nameOfRegionB, boolean forcible) {
-    throw new NotImplementedException("mergeRegions not supported in ThriftAdmin");
-
-  }
-
-  @Override
   public Future<Void> mergeRegionsAsync(byte[] nameOfRegionA, byte[] nameOfRegionB,
       boolean forcible) {
     throw new NotImplementedException("mergeRegionsAsync not supported in ThriftAdmin");
@@ -833,31 +661,16 @@ public class ThriftAdmin implements Admin {
   }
 
   @Override
-  public void splitRegion(byte[] regionName) {
-    throw new NotImplementedException("splitRegion not supported in ThriftAdmin");
-  }
-
-  @Override
   public void split(TableName tableName, byte[] splitPoint) {
     throw new NotImplementedException("split not supported in ThriftAdmin");
   }
 
   @Override
-  public void splitRegion(byte[] regionName, byte[] splitPoint) {
-    throw new NotImplementedException("splitRegion not supported in ThriftAdmin");
-  }
-
-  @Override
   public Future<Void> splitRegionAsync(byte[] regionName, byte[] splitPoint) {
     throw new NotImplementedException("splitRegionAsync not supported in ThriftAdmin");
   }
 
   @Override
-  public Future<Void> modifyTableAsync(TableName tableName, TableDescriptor td) {
-    throw new NotImplementedException("modifyTableAsync not supported in ThriftAdmin");
-  }
-
-  @Override
   public Future<Void> modifyTableAsync(TableDescriptor td) {
     throw new NotImplementedException("modifyTableAsync not supported in ThriftAdmin");
   }
@@ -906,11 +719,6 @@ public class ThriftAdmin implements Admin {
   }
 
   @Override
-  public List<HRegionInfo> getTableRegions(TableName tableName) {
-    throw new NotImplementedException("getTableRegions not supported in ThriftAdmin");
-  }
-
-  @Override
   public List<RegionInfo> getRegions(TableName tableName) {
     throw new NotImplementedException("getRegions not supported in ThriftAdmin");
   }
@@ -975,12 +783,6 @@ public class ThriftAdmin implements Admin {
   }
 
   @Override
-  public void snapshot(byte[] snapshotName, TableName tableName) {
-    throw new NotImplementedException("snapshot not supported in ThriftAdmin");
-
-  }
-
-  @Override
   public void snapshot(String snapshotName, TableName tableName, SnapshotType type) {
     throw new NotImplementedException("snapshot not supported in ThriftAdmin");
 
@@ -1004,22 +806,10 @@ public class ThriftAdmin implements Admin {
   }
 
   @Override
-  public void restoreSnapshot(byte[] snapshotName) {
-    throw new NotImplementedException("restoreSnapshot not supported in ThriftAdmin");
-
-  }
-
-  @Override
   public void restoreSnapshot(String snapshotName) {
     throw new NotImplementedException("restoreSnapshot not supported in ThriftAdmin");
 
   }
-
-  @Override
-  public Future<Void> restoreSnapshotAsync(String snapshotName) {
-    throw new NotImplementedException("restoreSnapshotAsync not supported in ThriftAdmin");
-  }
-
   @Override
   public void restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot,
       boolean restoreAcl) {
@@ -1053,24 +843,12 @@ public class ThriftAdmin implements Admin {
   public List<SnapshotDescription> listSnapshots() {
     throw new NotImplementedException("listSnapshots not supported in ThriftAdmin");
   }
-
-  @Override
-  public List<SnapshotDescription> listSnapshots(String regex) {
-    throw new NotImplementedException("listSnapshots not supported in ThriftAdmin");
-  }
-
   @Override
   public List<SnapshotDescription> listSnapshots(Pattern pattern) {
     throw new NotImplementedException("listSnapshots not supported in ThriftAdmin");
   }
 
   @Override
-  public List<SnapshotDescription> listTableSnapshots(String tableNameRegex,
-      String snapshotNameRegex) {
-    throw new NotImplementedException("listTableSnapshots not supported in ThriftAdmin");
-  }
-
-  @Override
   public List<SnapshotDescription> listTableSnapshots(Pattern tableNamePattern,
       Pattern snapshotNamePattern) {
     throw new NotImplementedException("listTableSnapshots not supported in ThriftAdmin");
@@ -1087,21 +865,11 @@ public class ThriftAdmin implements Admin {
   }
 
   @Override
-  public void deleteSnapshots(String regex) {
-    throw new NotImplementedException("deleteSnapshots not supported in ThriftAdmin");
-  }
-
-  @Override
   public void deleteSnapshots(Pattern pattern) {
     throw new NotImplementedException("deleteSnapshots not supported in ThriftAdmin");
   }
 
   @Override
-  public void deleteTableSnapshots(String tableNameRegex, String snapshotNameRegex) {
-    throw new NotImplementedException("deleteTableSnapshots not supported in ThriftAdmin");
-  }
-
-  @Override
   public void deleteTableSnapshots(Pattern tableNamePattern, Pattern snapshotNamePattern) {
     throw new NotImplementedException("deleteTableSnapshots not supported in ThriftAdmin");
   }
@@ -1112,11 +880,6 @@ public class ThriftAdmin implements Admin {
   }
 
   @Override
-  public QuotaRetriever getQuotaRetriever(QuotaFilter filter) {
-    throw new NotImplementedException("getQuotaRetriever not supported in ThriftAdmin");
-  }
-
-  @Override
   public List<QuotaSettings> getQuota(QuotaFilter filter) {
     throw new NotImplementedException("getQuota not supported in ThriftAdmin");
   }
@@ -1294,16 +1057,6 @@ public class ThriftAdmin implements Admin {
   }
 
   @Override
-  public Pair<Integer, Integer> getAlterStatus(TableName tableName) {
-    throw new NotImplementedException("getAlterStatus not supported in ThriftAdmin");
-  }
-
-  @Override
-  public Pair<Integer, Integer> getAlterStatus(byte[] tableName) {
-    throw new NotImplementedException("getAlterStatus not supported in ThriftAdmin");
-  }
-
-  @Override
   public Future<Void> deleteColumnFamilyAsync(TableName tableName, byte[] columnFamily) {
     throw new NotImplementedException("deleteColumnFamilyAsync not supported in ThriftAdmin");
   }


[hbase] 41/49: HBASE-22094: Throw TableNotFoundException if table not exists in AsyncAdmin.compact

Posted by op...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

openinx pushed a commit to branch HBASE-21879
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit cc85b6dc6530df83f24dc8555b83bb4a4652084b
Author: Sakthi <sa...@gmail.com>
AuthorDate: Mon Mar 25 22:54:48 2019 -0700

    HBASE-22094: Throw TableNotFoundException if table not exists in AsyncAdmin.compact
    
    Signed-off-by: zhangduo <zh...@apache.org>
---
 .../org/apache/hadoop/hbase/client/AsyncAdmin.java  | 12 ++++++++++++
 .../hadoop/hbase/client/RawAsyncHBaseAdmin.java     |  3 +++
 .../hbase/client/TestAsyncRegionAdminApi.java       | 21 +++++++++++++++++++++
 3 files changed, 36 insertions(+)

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 ea75f60..2f1f494 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
@@ -306,6 +306,7 @@ public interface AsyncAdmin {
   /**
    * Compact a table. When the returned CompletableFuture is done, it only means the compact request
    * was sent to HBase and may need some time to finish the compact operation.
+   * Throws {@link org.apache.hadoop.hbase.TableNotFoundException} if table not found.
    * @param tableName table to compact
    */
   default CompletableFuture<Void> compact(TableName tableName) {
@@ -316,6 +317,7 @@ public interface AsyncAdmin {
    * Compact a column family within a table. When the returned CompletableFuture is done, it only
    * means the compact request was sent to HBase and may need some time to finish the compact
    * operation.
+   * Throws {@link org.apache.hadoop.hbase.TableNotFoundException} if table not found.
    * @param tableName table to compact
    * @param columnFamily column family within a table. If not present, compact the table's all
    *          column families.
@@ -327,6 +329,8 @@ public interface AsyncAdmin {
   /**
    * Compact a table. When the returned CompletableFuture is done, it only means the compact request
    * was sent to HBase and may need some time to finish the compact operation.
+   * Throws {@link org.apache.hadoop.hbase.TableNotFoundException} if table not found for
+   * normal compaction type.
    * @param tableName table to compact
    * @param compactType {@link org.apache.hadoop.hbase.client.CompactType}
    */
@@ -336,6 +340,8 @@ public interface AsyncAdmin {
    * Compact a column family within a table. When the returned CompletableFuture is done, it only
    * means the compact request was sent to HBase and may need some time to finish the compact
    * operation.
+   * Throws {@link org.apache.hadoop.hbase.TableNotFoundException} if table not found for
+   * normal compaction type.
    * @param tableName table to compact
    * @param columnFamily column family within a table
    * @param compactType {@link org.apache.hadoop.hbase.client.CompactType}
@@ -363,6 +369,7 @@ public interface AsyncAdmin {
   /**
    * Major compact a table. When the returned CompletableFuture is done, it only means the compact
    * request was sent to HBase and may need some time to finish the compact operation.
+   * Throws {@link org.apache.hadoop.hbase.TableNotFoundException} if table not found.
    * @param tableName table to major compact
    */
   default CompletableFuture<Void> majorCompact(TableName tableName) {
@@ -373,6 +380,8 @@ public interface AsyncAdmin {
    * Major compact a column family within a table. When the returned CompletableFuture is done, it
    * only means the compact request was sent to HBase and may need some time to finish the compact
    * operation.
+   * Throws {@link org.apache.hadoop.hbase.TableNotFoundException} if table not found for
+   * normal compaction. type.
    * @param tableName table to major compact
    * @param columnFamily column family within a table. If not present, major compact the table's all
    *          column families.
@@ -384,6 +393,8 @@ public interface AsyncAdmin {
   /**
    * Major compact a table. When the returned CompletableFuture is done, it only means the compact
    * request was sent to HBase and may need some time to finish the compact operation.
+   * Throws {@link org.apache.hadoop.hbase.TableNotFoundException} if table not found for
+   * normal compaction type.
    * @param tableName table to major compact
    * @param compactType {@link org.apache.hadoop.hbase.client.CompactType}
    */
@@ -393,6 +404,7 @@ public interface AsyncAdmin {
    * Major compact a column family within a table. When the returned CompletableFuture is done, it
    * only means the compact request was sent to HBase and may need some time to finish the compact
    * operation.
+   * Throws {@link org.apache.hadoop.hbase.TableNotFoundException} if table not found.
    * @param tableName table to major compact
    * @param columnFamily column family within a table. If not present, major compact the table's all
    *          column families.
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 248e732..ef48f5a 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
@@ -1099,6 +1099,9 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
             future.completeExceptionally(err);
             return;
           }
+          if (locations == null || locations.isEmpty()) {
+            future.completeExceptionally(new TableNotFoundException(tableName));
+          }
           CompletableFuture<?>[] compactFutures =
             locations.stream().filter(l -> l.getRegion() != null)
               .filter(l -> !l.getRegion().isOffline()).filter(l -> l.getServerName() != null)
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi.java
index 6d30faf..3a8673c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi.java
@@ -36,6 +36,7 @@ import java.util.stream.Collectors;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.RegionState;
 import org.apache.hadoop.hbase.master.ServerManager;
@@ -427,6 +428,26 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
     }
   }
 
+  @Test
+  public void testNonExistentTableCompaction() {
+    testNonExistentTableCompaction(CompactionState.MINOR);
+    testNonExistentTableCompaction(CompactionState.MAJOR);
+  }
+
+  private void testNonExistentTableCompaction(CompactionState compactionState) {
+    try {
+      if (compactionState == CompactionState.MINOR) {
+        admin.compact(TableName.valueOf("NonExistentTable")).get();
+      } else {
+        admin.majorCompact(TableName.valueOf("NonExistentTable")).get();
+      }
+      fail("Expected TableNotFoundException when table doesn't exist");
+    } catch (Exception e) {
+      // expected.
+      assertTrue(e.getCause() instanceof TableNotFoundException);
+    }
+  }
+
   private static int countStoreFilesInFamily(List<Region> regions, final byte[] family) {
     return countStoreFilesInFamilies(regions, new byte[][] { family });
   }