You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zh...@apache.org on 2020/04/29 03:31:58 UTC

[hbase] branch branch-2 updated: HBASE-24249 Move code in FSHDFSUtils to FSUtils and mark related clas… (#1586)

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

zhangduo pushed a commit to branch branch-2
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/branch-2 by this push:
     new 922921e  HBASE-24249 Move code in FSHDFSUtils to FSUtils and mark related clas… (#1586)
922921e is described below

commit 922921ee5f40735e54c838718ee95ab27182f6e0
Author: Duo Zhang <zh...@apache.org>
AuthorDate: Wed Apr 29 10:44:34 2020 +0800

    HBASE-24249 Move code in FSHDFSUtils to FSUtils and mark related clas… (#1586)
    
    Signed-off-by: stack <st...@apache.org>
---
 .../apache/hadoop/hbase/util/CommonFSUtils.java    |   6 +-
 .../coprocessor/example/TestRefreshHFilesBase.java |   4 +-
 .../TestShadeSaslAuthenticationProvider.java       |   5 +-
 .../chaos/actions/RestartActiveNameNodeAction.java |   5 +-
 .../chaos/actions/RestartRandomDataNodeAction.java |   5 +-
 .../test/IntegrationTestZKAndFSPermissions.java    |   5 +-
 .../apache/hadoop/hbase/mapreduce/CopyTable.java   |  17 +-
 .../hadoop/hbase/mapreduce/HFileOutputFormat2.java |   6 +-
 .../MultiTableSnapshotInputFormatImpl.java         |  16 +-
 .../mapreduce/TableSnapshotInputFormatImpl.java    |  12 +-
 .../mapreduce/replication/VerifyReplication.java   |  22 +-
 .../hadoop/hbase/mob/mapreduce/MobRefReporter.java |   6 +-
 .../hadoop/hbase/regionserver/CompactionTool.java  |   3 +-
 .../hadoop/hbase/snapshot/ExportSnapshot.java      |  14 +-
 .../hadoop/hbase/ScanPerformanceEvaluation.java    |   7 +-
 .../TableSnapshotInputFormatTestBase.java          |   8 +-
 .../mapreduce/TestCellBasedHFileOutputFormat2.java |   7 +-
 .../hbase/mapreduce/TestCellBasedWALPlayer2.java   |   6 +-
 .../hbase/mapreduce/TestHFileOutputFormat2.java    |  12 +-
 .../TestMultiTableSnapshotInputFormat.java         |  12 +-
 .../TestMultiTableSnapshotInputFormatImpl.java     |  10 +-
 .../mapreduce/TestTableSnapshotInputFormat.java    |   4 +-
 .../hadoop/hbase/mapreduce/TestWALPlayer.java      |   6 +-
 .../hbase/mapreduce/TestWALRecordReader.java       |   4 +-
 .../hbase/regionserver/TestCompactionTool.java     |   4 +-
 .../hbase/replication/TestVerifyReplication.java   |  18 +-
 .../replication/TestVerifyReplicationAdjunct.java  |  14 +-
 .../TestVerifyReplicationCrossDiffHdfs.java        |   8 +-
 .../hadoop/hbase/snapshot/TestExportSnapshot.java  |   8 +-
 .../hbase/tmpl/master/MasterStatusTmpl.jamon       |   4 +-
 .../apache/hadoop/hbase/backup/HFileArchiver.java  |  15 +-
 .../example/LongTermArchivingHFileCleaner.java     |   5 +-
 .../hbase/client/ClientSideRegionScanner.java      |   7 +-
 .../hadoop/hbase/client/TableSnapshotScanner.java  |   5 +-
 .../org/apache/hadoop/hbase/fs/HFileSystem.java    |   4 +-
 .../java/org/apache/hadoop/hbase/io/FileLink.java  |  13 +-
 .../java/org/apache/hadoop/hbase/io/HFileLink.java |  15 +-
 .../io/asyncfs/FanOutOneBlockAsyncDFSOutput.java   |   7 +-
 .../FanOutOneBlockAsyncDFSOutputHelper.java        |   6 +-
 .../hadoop/hbase/io/hfile/HFilePrettyPrinter.java  |  16 +-
 .../hadoop/hbase/io/hfile/HFileWriterImpl.java     |   4 +-
 .../hadoop/hbase/master/CachedClusterId.java       |   4 +-
 .../apache/hadoop/hbase/master/CatalogJanitor.java |   9 +-
 .../hadoop/hbase/master/MasterFileSystem.java      |  16 +-
 .../hadoop/hbase/master/MasterWalManager.java      |  11 +-
 .../hadoop/hbase/master/SplitLogManager.java       |   8 +-
 .../hbase/master/assignment/GCRegionProcedure.java |   6 +-
 .../assignment/MergeTableRegionsProcedure.java     |   8 +-
 .../assignment/SplitTableRegionProcedure.java      |   4 +-
 .../hbase/master/cleaner/HFileLinkCleaner.java     |  18 +-
 .../AbstractStateMachineNamespaceProcedure.java    |   2 +-
 .../AbstractStateMachineTableProcedure.java        |   4 +-
 .../master/procedure/CloneSnapshotProcedure.java   |  11 +-
 .../master/procedure/CreateNamespaceProcedure.java |  12 +-
 .../master/procedure/CreateTableProcedure.java     |  10 +-
 .../master/procedure/DeleteNamespaceProcedure.java |  14 +-
 .../master/procedure/DeleteTableProcedure.java     |  12 +-
 .../hbase/master/procedure/MasterProcedureEnv.java |   3 +-
 .../snapshot/DisabledTableSnapshotHandler.java     |   7 +-
 .../master/snapshot/MasterSnapshotVerifier.java    |   4 +-
 .../hbase/master/snapshot/SnapshotFileCache.java   |  10 +-
 .../master/snapshot/SnapshotHFileCleaner.java      |  15 +-
 .../hbase/master/snapshot/SnapshotManager.java     |  11 +-
 .../hbase/master/snapshot/TakeSnapshotHandler.java |   5 +-
 .../hbase/mob/ManualMobMaintHFileCleaner.java      |   5 +-
 .../java/org/apache/hadoop/hbase/mob/MobUtils.java |   8 +-
 .../hadoop/hbase/mob/compactions/MobCompactor.java |   7 +-
 .../hbase/quotas/FileArchiverNotifierImpl.java     |   7 +-
 .../apache/hadoop/hbase/regionserver/HRegion.java  |  38 ++-
 .../hbase/regionserver/HRegionFileSystem.java      |  48 ++-
 .../hadoop/hbase/regionserver/HRegionServer.java   |   9 +-
 .../hbase/regionserver/SecureBulkLoadManager.java  |   8 +-
 .../hadoop/hbase/regionserver/SplitLogWorker.java  |  15 +-
 .../hadoop/hbase/regionserver/StoreFileWriter.java |   7 +-
 .../hbase/regionserver/wal/AbstractFSWAL.java      |   7 +-
 .../wal/AbstractProtobufLogWriter.java             |  24 +-
 .../hadoop/hbase/regionserver/wal/FSHLog.java      |   6 +-
 .../hadoop/hbase/regionserver/wal/ReaderBase.java  |   5 +-
 .../HBaseInterClusterReplicationEndpoint.java      |   6 +-
 .../replication/regionserver/HFileReplicator.java  |   6 +-
 .../regionserver/RecoveredReplicationSource.java   |   4 +-
 .../regionserver/ReplicationSyncUp.java            |   6 +-
 .../replication/regionserver/WALEntryStream.java   |   4 +-
 .../hbase/snapshot/RestoreSnapshotHelper.java      |  36 +-
 .../hbase/snapshot/SnapshotDescriptionUtils.java   |   9 +-
 .../apache/hadoop/hbase/snapshot/SnapshotInfo.java |  19 +-
 .../hadoop/hbase/snapshot/SnapshotManifest.java    |   9 +-
 .../hadoop/hbase/snapshot/SnapshotManifestV1.java  |   7 +-
 .../hadoop/hbase/snapshot/SnapshotManifestV2.java  |   7 +-
 .../hadoop/hbase/tool/HFileContentValidator.java   |   6 +-
 .../hadoop/hbase/tool/LoadIncrementalHFiles.java   |   4 +-
 .../org/apache/hadoop/hbase/util/FSHDFSUtils.java  | 332 ------------------
 .../hadoop/hbase/util/FSTableDescriptors.java      |  33 +-
 .../java/org/apache/hadoop/hbase/util/FSUtils.java | 375 +++++++++++++++++----
 .../org/apache/hadoop/hbase/util/FSVisitor.java    |   9 +-
 .../org/apache/hadoop/hbase/util/HBaseFsck.java    |  42 +--
 .../apache/hadoop/hbase/util/HBaseFsckRepair.java  |   2 +-
 .../apache/hadoop/hbase/util/HFileArchiveUtil.java |  19 +-
 .../apache/hadoop/hbase/util/HbckRegionInfo.java   |   3 +-
 .../apache/hadoop/hbase/util/RegionSplitter.java   |  14 +-
 .../util/compaction/MajorCompactionRequest.java    |   8 +-
 .../hbase/util/hbck/HFileCorruptionChecker.java    |  14 +-
 .../hadoop/hbase/wal/AbstractFSWALProvider.java    |  12 +-
 .../hadoop/hbase/wal/DisabledWALProvider.java      |   5 +-
 .../apache/hadoop/hbase/wal/WALPrettyPrinter.java  |   5 +-
 .../org/apache/hadoop/hbase/wal/WALSplitUtil.java  |  44 +--
 .../org/apache/hadoop/hbase/wal/WALSplitter.java   |   7 +-
 .../org/apache/hadoop/hbase/HBaseTestCase.java     |  10 +-
 .../apache/hadoop/hbase/HBaseTestingUtility.java   |   8 +-
 .../TestHColumnDescriptorDefaultVersions.java      |   4 +-
 .../org/apache/hadoop/hbase/TestZooKeeper.java     |   5 +-
 .../hadoop/hbase/backup/TestHFileArchiving.java    |  38 ++-
 .../example/TestZooKeeperTableArchiveClient.java   |  13 +-
 ...toreSnapshotFromClientSchemaChangeTestBase.java |   3 +-
 .../org/apache/hadoop/hbase/client/TestAdmin1.java |   4 +-
 .../hbase/client/TestAsyncTableAdminApi2.java      |  19 +-
 .../hadoop/hbase/client/TestFromClientSide5.java   |   3 +-
 .../hbase/client/TestSnapshotFromClient.java       |  26 +-
 .../client/TestSnapshotTemporaryDirectory.java     |  35 +-
 .../hbase/client/TestTableSnapshotScanner.java     |  13 +-
 .../hadoop/hbase/coprocessor/TestWALObserver.java  |  13 +-
 .../hbase/fs/TestBlockReorderMultiBlocks.java      |   6 +-
 .../hadoop/hbase/http/TestInfoServersACL.java      |  11 +-
 .../org/apache/hadoop/hbase/io/TestHFileLink.java  |  16 +-
 .../hbase/io/asyncfs/TestLocalAsyncOutput.java     |   6 +-
 .../hbase/io/hfile/TestHFilePrettyPrinter.java     |   4 +-
 .../hadoop/hbase/master/AbstractTestDLS.java       |  16 +-
 .../hadoop/hbase/master/TestCatalogJanitor.java    |  20 +-
 .../hadoop/hbase/master/TestMasterFileSystem.java  |  11 +-
 .../master/TestMasterFileSystemWithWALDir.java     |   8 +-
 .../hadoop/hbase/master/TestMasterNoCluster.java   |   5 +-
 .../hbase/master/TestMasterNotCarryTable.java      |   6 +-
 .../master/assignment/MockMasterServices.java      |   6 +-
 .../assignment/TestAssignmentManagerBase.java      |   4 +-
 .../hbase/master/assignment/TestHbckChore.java     |   4 +-
 .../hbase/master/cleaner/TestCleanerChore.java     |   6 +-
 .../hbase/master/cleaner/TestHFileLinkCleaner.java |  20 +-
 .../master/cleaner/TestSnapshotFromMaster.java     |  13 +-
 .../procedure/MasterProcedureTestingUtility.java   |  14 +-
 .../master/procedure/TestCreateTableProcedure.java |   3 +-
 .../TestDeleteColumnFamilyProcedureFromClient.java |   7 +-
 .../master/procedure/TestDeleteTableProcedure.java |   7 +-
 .../master/procedure/TestIgnoreUnknownFamily.java  |   3 +-
 .../TestMasterFailoverWithProcedures.java          |   4 +-
 .../TestTableDescriptorModificationFromClient.java |   5 +-
 .../procedure/TestTruncateTableProcedure.java      |   5 +-
 .../master/snapshot/TestSnapshotFileCache.java     |  14 +-
 .../master/snapshot/TestSnapshotHFileCleaner.java  |  11 +-
 .../apache/hadoop/hbase/mob/TestMobFileLink.java   |   8 +-
 .../compactions/TestPartitionedMobCompactor.java   |   6 +-
 .../hbase/namespace/TestNamespaceAuditor.java      |  10 +-
 .../hbase/quotas/TestFileArchiverNotifierImpl.java |   5 +-
 .../regionserver/TestBulkLoadReplication.java      |   5 +-
 .../regionserver/TestCacheOnWriteInSchema.java     |   6 +-
 .../hbase/regionserver/TestCleanupMetaWAL.java     |  10 +-
 .../hadoop/hbase/regionserver/TestClusterId.java   |   7 +-
 .../hbase/regionserver/TestCompactSplitThread.java |   4 +-
 .../TestCompactionArchiveConcurrentClose.java      |   9 +-
 .../TestCompactionArchiveIOException.java          |   8 +-
 .../hbase/regionserver/TestCompactionPolicy.java   |   7 +-
 .../hbase/regionserver/TestDeleteMobTable.java     |   5 +-
 .../hadoop/hbase/regionserver/TestHMobStore.java   |   9 +-
 .../hadoop/hbase/regionserver/TestHRegion.java     |  32 +-
 .../hbase/regionserver/TestHRegionFileSystem.java  |  13 +-
 .../hadoop/hbase/regionserver/TestHStore.java      |   6 +-
 .../hadoop/hbase/regionserver/TestHStoreFile.java  |  22 +-
 .../regionserver/TestHdfsSnapshotHRegion.java      |   8 +-
 .../hbase/regionserver/TestMobStoreCompaction.java |   5 +-
 .../hbase/regionserver/TestMobStoreScanner.java    |   6 +-
 .../hbase/regionserver/TestRecoveredEdits.java     |   4 +-
 .../TestRecoveredEditsReplayAndAbort.java          |   5 +-
 .../TestRegionMergeTransactionOnCluster.java       |   7 +-
 .../hadoop/hbase/regionserver/TestRegionOpen.java  |   4 +-
 .../regionserver/TestRowPrefixBloomFilter.java     |   5 +-
 .../TestSplitTransactionOnCluster.java             |   3 +-
 .../regionserver/TestStoreFileRefresherChore.java  |   9 +-
 .../TestWALMonotonicallyIncreasingSeqId.java       |   6 +-
 .../hbase/regionserver/wal/AbstractTestFSWAL.java  |   3 +-
 .../regionserver/wal/AbstractTestWALReplay.java    |  38 +--
 .../hbase/regionserver/wal/TestDurability.java     |   4 +-
 .../hadoop/hbase/regionserver/wal/TestFSHLog.java  |  11 +-
 .../hbase/regionserver/wal/TestLogRollAbort.java   |   6 +-
 .../hbase/regionserver/wal/TestLogRolling.java     |   8 +-
 .../regionserver/wal/TestLogRollingNoCluster.java  |   4 +-
 .../regionserver/wal/TestWALActionsListener.java   |   6 +-
 .../regionserver/TestReplicationSink.java          |   4 +-
 .../regionserver/TestReplicationSourceManager.java |   4 +-
 .../TestCustomSaslAuthenticationProvider.java      |   7 +-
 .../hbase/security/token/SecureTestCluster.java    |   7 +-
 .../hbase/snapshot/SnapshotTestingUtils.java       |  10 +-
 .../hbase/snapshot/TestRegionSnapshotTask.java     |   4 +-
 .../hbase/snapshot/TestRestoreSnapshotHelper.java  |  29 +-
 .../hbase/snapshot/TestSnapshotStoreFileSize.java  |   7 +-
 .../snapshot/TestSnapshotWhenChoreCleaning.java    |   9 +-
 .../hbase/tool/TestLoadIncrementalHFiles.java      |   6 +-
 .../TestLoadIncrementalHFilesSplitRecovery.java    |   4 +-
 .../hadoop/hbase/util/BaseTestHBaseFsck.java       |  18 +-
 .../hadoop/hbase/util/HFileArchiveTestingUtil.java |   7 +-
 .../apache/hadoop/hbase/util/TestFSHDFSUtils.java  |  44 +--
 .../hadoop/hbase/util/TestFSTableDescriptors.java  |  14 +-
 .../org/apache/hadoop/hbase/util/TestFSUtils.java  |  65 ++--
 .../apache/hadoop/hbase/util/TestFSVisitor.java    |   3 +-
 .../apache/hadoop/hbase/util/TestHBaseFsckMOB.java |   2 +-
 .../hadoop/hbase/util/TestHFileArchiveUtil.java    |   8 +-
 .../org/apache/hadoop/hbase/util/TestRootPath.java |   6 +-
 .../util/hbck/OfflineMetaRebuildTestCore.java      |  10 +-
 .../wal/TestBoundedRegionGroupingStrategy.java     |   4 +-
 .../hadoop/hbase/wal/TestFSHLogProvider.java       |  10 +-
 .../hadoop/hbase/wal/TestReadWriteSeqIdFiles.java  |   4 +-
 .../org/apache/hadoop/hbase/wal/TestSecureWAL.java |   4 +-
 .../apache/hadoop/hbase/wal/TestWALFactory.java    |  14 +-
 .../apache/hadoop/hbase/wal/TestWALMethods.java    |   4 +-
 .../hbase/wal/TestWALOpenAfterDNRollingStart.java  |   4 +-
 .../hadoop/hbase/wal/TestWALReaderOnSecureWAL.java |   9 +-
 .../apache/hadoop/hbase/wal/TestWALRootDir.java    |   6 +-
 .../org/apache/hadoop/hbase/wal/TestWALSplit.java  |  16 +-
 .../hadoop/hbase/wal/TestWALSplitToHFile.java      |  12 +-
 .../hadoop/hbase/wal/WALPerformanceEvaluation.java |   9 +-
 218 files changed, 1403 insertions(+), 1504 deletions(-)

diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CommonFSUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CommonFSUtils.java
index 27e66ab..2e21605 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CommonFSUtils.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CommonFSUtils.java
@@ -44,6 +44,7 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.yetus.audience.InterfaceAudience;
 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;
 
@@ -55,7 +56,7 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
  * <a href="https://issues.apache.org/jira/browse/HBASE-20838">HBASE-20838</a> for more details.
  */
 @InterfaceAudience.Private
-public abstract class CommonFSUtils {
+public final class CommonFSUtils {
   private static final Logger LOG = LoggerFactory.getLogger(CommonFSUtils.class);
 
   /** Parameter name for HBase WAL directory */
@@ -68,8 +69,7 @@ public abstract class CommonFSUtils {
   /** Full access permissions (starting point for a umask) */
   public static final String FULL_RWX_PERMISSIONS = "777";
 
-  protected CommonFSUtils() {
-    super();
+  private CommonFSUtils() {
   }
 
   /**
diff --git a/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestRefreshHFilesBase.java b/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestRefreshHFilesBase.java
index e84929d..e8cab77 100644
--- a/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestRefreshHFilesBase.java
+++ b/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestRefreshHFilesBase.java
@@ -29,7 +29,7 @@ import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.master.MasterFileSystem;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.HFileTestUtil;
 import org.junit.After;
 import org.slf4j.Logger;
@@ -77,7 +77,7 @@ public class TestRefreshHFilesBase {
 
   protected void addHFilesToRegions() throws IOException {
     MasterFileSystem mfs = HTU.getMiniHBaseCluster().getMaster().getMasterFileSystem();
-    Path tableDir = FSUtils.getTableDir(mfs.getRootDir(), TABLE_NAME);
+    Path tableDir = CommonFSUtils.getTableDir(mfs.getRootDir(), TABLE_NAME);
     for (Region region : cluster.getRegions(TABLE_NAME)) {
       Path regionDir = new Path(tableDir, region.getRegionInfo().getEncodedName());
       Path familyDir = new Path(regionDir, Bytes.toString(FAMILY));
diff --git a/hbase-examples/src/test/java/org/apache/hadoop/hbase/security/provider/example/TestShadeSaslAuthenticationProvider.java b/hbase-examples/src/test/java/org/apache/hadoop/hbase/security/provider/example/TestShadeSaslAuthenticationProvider.java
index 001842f..3625bf4 100644
--- a/hbase-examples/src/test/java/org/apache/hadoop/hbase/security/provider/example/TestShadeSaslAuthenticationProvider.java
+++ b/hbase-examples/src/test/java/org/apache/hadoop/hbase/security/provider/example/TestShadeSaslAuthenticationProvider.java
@@ -30,7 +30,6 @@ import java.security.PrivilegedExceptionAction;
 import java.util.Collections;
 import java.util.Map;
 import java.util.Map.Entry;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -60,7 +59,7 @@ import org.apache.hadoop.hbase.security.token.TokenProvider;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.SecurityTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.minikdc.MiniKdc;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.junit.AfterClass;
@@ -104,7 +103,7 @@ public class TestShadeSaslAuthenticationProvider {
         USER_DATABASE_FILE.toString());
 
     Path rootdir = new Path(testDir, "hbase-root");
-    FSUtils.setRootDir(CONF, rootdir);
+    CommonFSUtils.setRootDir(CONF, rootdir);
     LocalHBaseCluster cluster = new LocalHBaseCluster(CONF, 1);
     return cluster;
   }
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartActiveNameNodeAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartActiveNameNodeAction.java
index 2fc64e6..672169e 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartActiveNameNodeAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartActiveNameNodeAction.java
@@ -19,10 +19,9 @@
 package org.apache.hadoop.hbase.chaos.actions;
 
 import java.util.List;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
@@ -55,7 +54,7 @@ public class RestartActiveNameNodeAction extends RestartActionBaseAction {
   @Override
   public void perform() throws Exception {
     LOG.info("Performing action: Restart active namenode");
-    Configuration conf = FSUtils.getRootDir(getConf()).getFileSystem(getConf()).getConf();
+    Configuration conf = CommonFSUtils.getRootDir(getConf()).getFileSystem(getConf()).getConf();
     String nameServiceID = DFSUtil.getNamenodeNameServiceId(conf);
     if (!HAUtil.isHAEnabled(conf, nameServiceID)) {
       throw new Exception("HA for namenode is not enabled");
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRandomDataNodeAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRandomDataNodeAction.java
index 5824a42..81920c8 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRandomDataNodeAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRandomDataNodeAction.java
@@ -21,10 +21,9 @@ package org.apache.hadoop.hbase.chaos.actions;
 import java.io.IOException;
 import java.util.LinkedList;
 import java.util.List;
-
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -50,7 +49,7 @@ public class RestartRandomDataNodeAction extends RestartActionBaseAction {
   }
 
   public ServerName[] getDataNodes() throws IOException {
-    DistributedFileSystem fs = (DistributedFileSystem) FSUtils.getRootDir(getConf())
+    DistributedFileSystem fs = (DistributedFileSystem) CommonFSUtils.getRootDir(getConf())
         .getFileSystem(getConf());
     DFSClient dfsClient = fs.getClient();
     List<ServerName> hosts = new LinkedList<>();
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestZKAndFSPermissions.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestZKAndFSPermissions.java
index 866f017..f752743 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestZKAndFSPermissions.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestZKAndFSPermissions.java
@@ -24,7 +24,6 @@ import static org.junit.Assert.fail;
 
 import java.io.IOException;
 import java.util.List;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -34,7 +33,7 @@ import org.apache.hadoop.hbase.IntegrationTestingUtility;
 import org.apache.hadoop.hbase.security.SecurityConstants;
 import org.apache.hadoop.hbase.testclassification.IntegrationTests;
 import org.apache.hadoop.hbase.util.AbstractHBaseTool;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
@@ -217,7 +216,7 @@ public class IntegrationTestZKAndFSPermissions extends AbstractHBaseTool {
   }
 
   private void testFSPerms() throws IOException {
-    Path rootDir = FSUtils.getRootDir(conf);
+    Path rootDir = CommonFSUtils.getRootDir(conf);
 
     LOG.info("");
     LOG.info("***********************************************************************************");
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/CopyTable.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/CopyTable.java
index 413ac30..bde3519 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/CopyTable.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/CopyTable.java
@@ -22,28 +22,27 @@ import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.UUID;
-
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.mapreduce.Import.CellImporter;
-import org.apache.hadoop.hbase.mapreduce.Import.Importer;
-import org.apache.hadoop.hbase.util.FSUtils;
-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.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.mapreduce.Import.CellImporter;
+import org.apache.hadoop.hbase.mapreduce.Import.Importer;
 import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Tool used to copy a table to another one which can be on a different setup.
@@ -78,7 +77,7 @@ public class CopyTable extends Configured implements Tool {
   private final static String JOB_NAME_CONF_KEY = "mapreduce.job.name";
 
   private Path generateUniqTempDir(boolean withDirCreated) throws IOException {
-    FileSystem fs = FSUtils.getCurrentFileSystem(getConf());
+    FileSystem fs = CommonFSUtils.getCurrentFileSystem(getConf());
     Path dir = new Path(fs.getWorkingDirectory(), NAME);
     if (!fs.exists(dir)) {
       fs.mkdirs(dir);
@@ -423,7 +422,7 @@ public class CopyTable extends Configured implements Tool {
       if (code == 0) {
         // bulkloadDir is deleted only LoadIncrementalHFiles was successful so that one can rerun
         // LoadIncrementalHFiles.
-        FileSystem fs = FSUtils.getCurrentFileSystem(getConf());
+        FileSystem fs = CommonFSUtils.getCurrentFileSystem(getConf());
         if (!fs.delete(this.bulkloadDir, true)) {
           LOG.error("Deleting folder " + bulkloadDir + " failed!");
           code = 1;
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 e48861f..a1012b3 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
@@ -21,6 +21,7 @@ import static org.apache.hadoop.hbase.regionserver.HStoreFile.BULKLOAD_TASK_KEY;
 import static org.apache.hadoop.hbase.regionserver.HStoreFile.BULKLOAD_TIME_KEY;
 import static org.apache.hadoop.hbase.regionserver.HStoreFile.EXCLUDE_FROM_MINOR_COMPACTION_KEY;
 import static org.apache.hadoop.hbase.regionserver.HStoreFile.MAJOR_COMPACTION_KEY;
+
 import java.io.IOException;
 import java.io.UnsupportedEncodingException;
 import java.net.InetSocketAddress;
@@ -71,8 +72,8 @@ import org.apache.hadoop.hbase.regionserver.HStore;
 import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
 import org.apache.hadoop.hbase.util.BloomFilterUtil;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.MapReduceExtendedCell;
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.io.SequenceFile;
@@ -88,6 +89,7 @@ import org.apache.hadoop.mapreduce.lib.partition.TotalOrderPartitioner;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 
 /**
@@ -443,7 +445,7 @@ public class HFileOutputFormat2
     String policy =
         conf.get(STORAGE_POLICY_PROPERTY_CF_PREFIX + Bytes.toString(tableAndFamily),
           conf.get(STORAGE_POLICY_PROPERTY));
-    FSUtils.setStoragePolicy(fs, cfPath, policy);
+    CommonFSUtils.setStoragePolicy(fs, cfPath, policy);
   }
 
   /*
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableSnapshotInputFormatImpl.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableSnapshotInputFormatImpl.java
index 8a94de5..ef5c161 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableSnapshotInputFormatImpl.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableSnapshotInputFormatImpl.java
@@ -24,7 +24,6 @@ import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 import java.util.UUID;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -32,8 +31,8 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper;
 import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.ConfigurationUtil;
-import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.yetus.audience.InterfaceStability;
 import org.slf4j.Logger;
@@ -62,16 +61,12 @@ public class MultiTableSnapshotInputFormatImpl {
   /**
    * Configure conf to read from snapshotScans, with snapshots restored to a subdirectory of
    * restoreDir.
+   * <p/>
    * Sets: {@link #RESTORE_DIRS_KEY}, {@link #SNAPSHOT_TO_SCANS_KEY}
-   *
-   * @param conf
-   * @param snapshotScans
-   * @param restoreDir
-   * @throws IOException
    */
   public void setInput(Configuration conf, Map<String, Collection<Scan>> snapshotScans,
       Path restoreDir) throws IOException {
-    Path rootDir = FSUtils.getRootDir(conf);
+    Path rootDir = CommonFSUtils.getRootDir(conf);
     FileSystem fs = rootDir.getFileSystem(conf);
 
     setSnapshotToScans(conf, snapshotScans);
@@ -92,7 +87,7 @@ public class MultiTableSnapshotInputFormatImpl {
    */
   public List<TableSnapshotInputFormatImpl.InputSplit> getSplits(Configuration conf)
       throws IOException {
-    Path rootDir = FSUtils.getRootDir(conf);
+    Path rootDir = CommonFSUtils.getRootDir(conf);
     FileSystem fs = rootDir.getFileSystem(conf);
 
     List<TableSnapshotInputFormatImpl.InputSplit> rtn = Lists.newArrayList();
@@ -229,12 +224,11 @@ public class MultiTableSnapshotInputFormatImpl {
    * @param conf          configuration to restore with
    * @param snapshotToDir mapping from snapshot names to restore directories
    * @param fs            filesystem to do snapshot restoration on
-   * @throws IOException
    */
   public void restoreSnapshots(Configuration conf, Map<String, Path> snapshotToDir, FileSystem fs)
       throws IOException {
     // TODO: restore from record readers to parallelize.
-    Path rootDir = FSUtils.getRootDir(conf);
+    Path rootDir = CommonFSUtils.getRootDir(conf);
 
     for (Map.Entry<String, Path> entry : snapshotToDir.entrySet()) {
       String snapshotName = entry.getKey();
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormatImpl.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormatImpl.java
index 73b5d05..9758f15 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormatImpl.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormatImpl.java
@@ -26,7 +26,6 @@ import java.lang.reflect.InvocationTargetException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.UUID;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -45,7 +44,7 @@ import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper;
 import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
 import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.RegionSplitter;
 import org.apache.hadoop.io.Writable;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -53,6 +52,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.TableSnapshotRegionSplit;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
@@ -223,7 +223,7 @@ public class TableSnapshotInputFormatImpl {
       this.split = split;
       TableDescriptor htd = split.htd;
       HRegionInfo hri = this.split.getRegionInfo();
-      FileSystem fs = FSUtils.getCurrentFileSystem(conf);
+      FileSystem fs = CommonFSUtils.getCurrentFileSystem(conf);
 
 
       // region is immutable, this should be fine,
@@ -277,7 +277,7 @@ public class TableSnapshotInputFormatImpl {
   public static List<InputSplit> getSplits(Configuration conf) throws IOException {
     String snapshotName = getSnapshotName(conf);
 
-    Path rootDir = FSUtils.getRootDir(conf);
+    Path rootDir = CommonFSUtils.getRootDir(conf);
     FileSystem fs = rootDir.getFileSystem(conf);
 
     SnapshotManifest manifest = getSnapshotManifest(conf, snapshotName, rootDir, fs);
@@ -363,7 +363,7 @@ public class TableSnapshotInputFormatImpl {
     // load table descriptor
     TableDescriptor htd = manifest.getTableDescriptor();
 
-    Path tableDir = FSUtils.getTableDir(restoreDir, htd.getTableName());
+    Path tableDir = CommonFSUtils.getTableDir(restoreDir, htd.getTableName());
 
     boolean localityEnabled = conf.getBoolean(SNAPSHOT_INPUTFORMAT_LOCALITY_ENABLED_KEY,
                                               SNAPSHOT_INPUTFORMAT_LOCALITY_ENABLED_DEFAULT);
@@ -533,7 +533,7 @@ public class TableSnapshotInputFormatImpl {
       conf.set(SPLIT_ALGO, splitAlgo.getClass().getName());
     }
     conf.setInt(NUM_SPLITS_PER_REGION, numSplitsPerRegion);
-    Path rootDir = FSUtils.getRootDir(conf);
+    Path rootDir = CommonFSUtils.getRootDir(conf);
     FileSystem fs = rootDir.getFileSystem(conf);
 
     restoreDir = new Path(restoreDir, UUID.randomUUID().toString());
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
index 6b200a2..7f8a71e 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
@@ -54,7 +54,7 @@ import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
 import org.apache.hadoop.hbase.replication.ReplicationUtils;
 import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.zookeeper.ZKConfig;
@@ -216,13 +216,13 @@ public class VerifyReplication extends Configured implements Tool {
           String peerFSAddress = conf.get(NAME + ".peerFSAddress", null);
           String peerHBaseRootAddress = conf.get(NAME + ".peerHBaseRootAddress", null);
           FileSystem.setDefaultUri(peerConf, peerFSAddress);
-          FSUtils.setRootDir(peerConf, new Path(peerHBaseRootAddress));
-          LOG.info("Using peer snapshot:" + peerSnapshotName + " with temp dir:"
-              + peerSnapshotTmpDir + " peer root uri:" + FSUtils.getRootDir(peerConf)
-              + " peerFSAddress:" + peerFSAddress);
+          CommonFSUtils.setRootDir(peerConf, new Path(peerHBaseRootAddress));
+          LOG.info("Using peer snapshot:" + peerSnapshotName + " with temp dir:" +
+            peerSnapshotTmpDir + " peer root uri:" + CommonFSUtils.getRootDir(peerConf) +
+            " peerFSAddress:" + peerFSAddress);
 
-          replicatedScanner = new TableSnapshotScanner(peerConf, FSUtils.getRootDir(peerConf),
-              new Path(peerFSAddress, peerSnapshotTmpDir), peerSnapshotName, scan, true);
+          replicatedScanner = new TableSnapshotScanner(peerConf, CommonFSUtils.getRootDir(peerConf),
+            new Path(peerFSAddress, peerSnapshotTmpDir), peerSnapshotName, scan, true);
         } else {
           replicatedScanner = replicatedTable.getScanner(scan);
         }
@@ -367,13 +367,13 @@ public class VerifyReplication extends Configured implements Tool {
   }
 
   private void restoreSnapshotForPeerCluster(Configuration conf, String peerQuorumAddress)
-      throws IOException {
+    throws IOException {
     Configuration peerConf =
-        HBaseConfiguration.createClusterConf(conf, peerQuorumAddress, PEER_CONFIG_PREFIX);
+      HBaseConfiguration.createClusterConf(conf, peerQuorumAddress, PEER_CONFIG_PREFIX);
     FileSystem.setDefaultUri(peerConf, peerFSAddress);
-    FSUtils.setRootDir(peerConf, new Path(peerFSAddress, peerHBaseRootAddress));
+    CommonFSUtils.setRootDir(peerConf, new Path(peerFSAddress, peerHBaseRootAddress));
     FileSystem fs = FileSystem.get(peerConf);
-    RestoreSnapshotHelper.copySnapshotForScanner(peerConf, fs, FSUtils.getRootDir(peerConf),
+    RestoreSnapshotHelper.copySnapshotForScanner(peerConf, fs, CommonFSUtils.getRootDir(peerConf),
       new Path(peerFSAddress, peerSnapshotTmpDir), peerSnapshotName);
   }
 
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mob/mapreduce/MobRefReporter.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mob/mapreduce/MobRefReporter.java
index 929d64a..4152182 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mob/mapreduce/MobRefReporter.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mob/mapreduce/MobRefReporter.java
@@ -24,7 +24,6 @@ import java.util.Base64;
 import java.util.HashSet;
 import java.util.Set;
 import java.util.UUID;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.FileStatus;
@@ -49,8 +48,8 @@ import org.apache.hadoop.hbase.mapreduce.TableMapper;
 import org.apache.hadoop.hbase.mob.MobConstants;
 import org.apache.hadoop.hbase.mob.MobUtils;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.HFileArchiveUtil;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.io.Text;
@@ -61,7 +60,6 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.yetus.audience.InterfaceAudience;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -267,7 +265,7 @@ public class MobRefReporter extends Configured implements Tool {
 
         Path backRefDir = HFileLink.getBackReferencesDir(archive, file);
         try {
-          FileStatus[] backRefs = FSUtils.listStatus(archive.getFileSystem(conf), backRefDir);
+          FileStatus[] backRefs = CommonFSUtils.listStatus(archive.getFileSystem(conf), backRefDir);
           if (backRefs != null) {
             boolean found = false;
             for (FileStatus backRef : backRefs) {
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java
index f01e3d9..49b9d34 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java
@@ -26,7 +26,6 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.Optional;
 import java.util.Set;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -271,7 +270,7 @@ public class CompactionTool extends Configured implements Tool {
      */
     private static String[] getStoreDirHosts(final FileSystem fs, final Path path)
         throws IOException {
-      FileStatus[] files = FSUtils.listStatus(fs, path);
+      FileStatus[] files = CommonFSUtils.listStatus(fs, path);
       if (files == null) {
         return new String[] {};
       }
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java
index 198ae97..c215243 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java
@@ -34,7 +34,6 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 import java.util.function.BiConsumer;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -54,6 +53,7 @@ import org.apache.hadoop.hbase.io.hadoopbackport.ThrottledInputStream;
 import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
 import org.apache.hadoop.hbase.mob.MobUtils;
 import org.apache.hadoop.hbase.util.AbstractHBaseTool;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.HFileArchiveUtil;
 import org.apache.hadoop.hbase.util.Pair;
@@ -75,8 +75,10 @@ import org.apache.hadoop.util.Tool;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
 import org.apache.hbase.thirdparty.org.apache.commons.cli.Option;
+
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotFileInfo;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
@@ -255,7 +257,7 @@ public class ExportSnapshot extends AbstractHBaseTool implements Tool {
           TableName table =HFileLink.getReferencedTableName(inputPath.getName());
           String region = HFileLink.getReferencedRegionName(inputPath.getName());
           String hfile = HFileLink.getReferencedHFileName(inputPath.getName());
-          path = new Path(FSUtils.getTableDir(new Path("./"), table),
+          path = new Path(CommonFSUtils.getTableDir(new Path("./"), table),
               new Path(region, new Path(family, hfile)));
           break;
         case WAL:
@@ -845,8 +847,8 @@ public class ExportSnapshot extends AbstractHBaseTool implements Tool {
       final FileSystem fs, final Path rootDir, final Path snapshotDir) throws IOException {
     // Update the conf with the current root dir, since may be a different cluster
     Configuration conf = new Configuration(baseConf);
-    FSUtils.setRootDir(conf, rootDir);
-    FSUtils.setFsDefault(conf, FSUtils.getRootDir(conf));
+    CommonFSUtils.setRootDir(conf, rootDir);
+    CommonFSUtils.setFsDefault(conf, CommonFSUtils.getRootDir(conf));
     SnapshotDescription snapshotDesc = SnapshotDescriptionUtils.readSnapshotInfo(fs, snapshotDir);
     SnapshotReferenceUtil.verifySnapshot(conf, fs, snapshotDir, snapshotDesc);
   }
@@ -959,9 +961,9 @@ public class ExportSnapshot extends AbstractHBaseTool implements Tool {
       targetName = snapshotName;
     }
     if (inputRoot == null) {
-      inputRoot = FSUtils.getRootDir(conf);
+      inputRoot = CommonFSUtils.getRootDir(conf);
     } else {
-      FSUtils.setRootDir(conf, inputRoot);
+      CommonFSUtils.setRootDir(conf, inputRoot);
     }
 
     Configuration srcConf = HBaseConfiguration.createClusterConf(conf, null, CONF_SOURCE_PREFIX);
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/ScanPerformanceEvaluation.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/ScanPerformanceEvaluation.java
index c181b19..ca28130 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/ScanPerformanceEvaluation.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/ScanPerformanceEvaluation.java
@@ -20,12 +20,10 @@ package org.apache.hadoop.hbase;
 
 import java.io.IOException;
 import java.util.concurrent.TimeUnit;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Result;
@@ -38,13 +36,14 @@ import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
 import org.apache.hadoop.hbase.mapreduce.TableMapper;
 import org.apache.hadoop.hbase.util.AbstractHBaseTool;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.mapreduce.Counters;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.ToolRunner;
+import org.apache.yetus.audience.InterfaceAudience;
 
 import org.apache.hbase.thirdparty.com.google.common.base.Stopwatch;
 import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
@@ -70,7 +69,7 @@ public class ScanPerformanceEvaluation extends AbstractHBaseTool {
     super.setConf(conf);
     Path rootDir;
     try {
-      rootDir = FSUtils.getRootDir(conf);
+      rootDir = CommonFSUtils.getRootDir(conf);
       rootDir.getFileSystem(conf);
     } catch (IOException ex) {
       throw new RuntimeException(ex);
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormatTestBase.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormatTestBase.java
index 95250d2..a765f5f 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormatTestBase.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormatTestBase.java
@@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
 import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.HFileArchiveUtil;
 import org.junit.Assert;
@@ -126,8 +127,9 @@ public abstract class TableSnapshotInputFormatTestBase {
 
       testRestoreSnapshotDoesNotCreateBackRefLinksInit(tableName, snapshotName,tmpTableDir);
 
-      Path rootDir = FSUtils.getRootDir(UTIL.getConfiguration());
-      for (Path regionDir : FSUtils.getRegionDirs(fs, FSUtils.getTableDir(rootDir, tableName))) {
+      Path rootDir = CommonFSUtils.getRootDir(UTIL.getConfiguration());
+      for (Path regionDir : FSUtils.getRegionDirs(fs,
+        CommonFSUtils.getTableDir(rootDir, tableName))) {
         for (Path storeDir : FSUtils.getFamilyDirs(fs, regionDir)) {
           for (FileStatus status : fs.listStatus(storeDir)) {
             System.out.println(status.getPath());
@@ -212,7 +214,7 @@ public abstract class TableSnapshotInputFormatTestBase {
     Table table = util.getConnection().getTable(tableName);
     util.loadTable(table, FAMILIES);
 
-    Path rootDir = FSUtils.getRootDir(util.getConfiguration());
+    Path rootDir = CommonFSUtils.getRootDir(util.getConfiguration());
     FileSystem fs = rootDir.getFileSystem(util.getConfiguration());
 
     LOG.info("snapshot");
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellBasedHFileOutputFormat2.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellBasedHFileOutputFormat2.java
index dd243f9..ed151ec 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellBasedHFileOutputFormat2.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellBasedHFileOutputFormat2.java
@@ -89,6 +89,7 @@ import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.VerySlowMapReduceTests;
 import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.ReflectionUtils;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
@@ -1150,7 +1151,7 @@ public class TestCellBasedHFileOutputFormat2  {
       // commit so that the filesystem has one directory per column family
       hof.getOutputCommitter(context).commitTask(context);
       hof.getOutputCommitter(context).commitJob(context);
-      FileStatus[] families = FSUtils.listStatus(fs, dir, new FSUtils.FamilyDirFilter(fs));
+      FileStatus[] families = CommonFSUtils.listStatus(fs, dir, new FSUtils.FamilyDirFilter(fs));
       assertEquals(htd.getFamilies().size(), families.length);
       for (FileStatus f : families) {
         String familyStr = f.getPath().getName();
@@ -1224,7 +1225,7 @@ public class TestCellBasedHFileOutputFormat2  {
 
       // deep inspection: get the StoreFile dir
       final Path storePath = new Path(
-        FSUtils.getTableDir(FSUtils.getRootDir(conf), TABLE_NAMES[0]),
+        CommonFSUtils.getTableDir(CommonFSUtils.getRootDir(conf), TABLE_NAMES[0]),
           new Path(admin.getTableRegions(TABLE_NAMES[0]).get(0).getEncodedName(),
             Bytes.toString(FAMILIES[0])));
       assertEquals(0, fs.listStatus(storePath).length);
@@ -1305,7 +1306,7 @@ public class TestCellBasedHFileOutputFormat2  {
 
       // deep inspection: get the StoreFile dir
       final Path storePath = new Path(
-        FSUtils.getTableDir(FSUtils.getRootDir(conf), TABLE_NAMES[0]),
+        CommonFSUtils.getTableDir(CommonFSUtils.getRootDir(conf), TABLE_NAMES[0]),
           new Path(admin.getTableRegions(TABLE_NAMES[0]).get(0).getEncodedName(),
             Bytes.toString(FAMILIES[0])));
       assertEquals(0, fs.listStatus(storePath).length);
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellBasedWALPlayer2.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellBasedWALPlayer2.java
index 9a63ae9..71bd7c1 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellBasedWALPlayer2.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellBasedWALPlayer2.java
@@ -49,7 +49,7 @@ import org.apache.hadoop.hbase.mapreduce.WALPlayer.WALCellMapper;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MapReduceTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.LauncherSecurityManager;
 import org.apache.hadoop.hbase.util.MapReduceExtendedCell;
 import org.apache.hadoop.hbase.wal.WAL;
@@ -94,8 +94,8 @@ public class TestCellBasedWALPlayer2 {
     conf= TEST_UTIL.getConfiguration();
     rootDir = TEST_UTIL.createRootDir();
     walRootDir = TEST_UTIL.createWALRootDir();
-    fs = FSUtils.getRootDirFileSystem(conf);
-    logFs = FSUtils.getWALFileSystem(conf);
+    fs = CommonFSUtils.getRootDirFileSystem(conf);
+    logFs = CommonFSUtils.getWALFileSystem(conf);
     cluster = TEST_UTIL.startMiniCluster();
   }
 
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 30b0022..3d7ce19 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
@@ -89,6 +89,7 @@ import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.VerySlowMapReduceTests;
 import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.ReflectionUtils;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
@@ -1169,7 +1170,7 @@ public class TestHFileOutputFormat2  {
       // commit so that the filesystem has one directory per column family
       hof.getOutputCommitter(context).commitTask(context);
       hof.getOutputCommitter(context).commitJob(context);
-      FileStatus[] families = FSUtils.listStatus(fs, dir, new FSUtils.FamilyDirFilter(fs));
+      FileStatus[] families = CommonFSUtils.listStatus(fs, dir, new FSUtils.FamilyDirFilter(fs));
       assertEquals(htd.getFamilies().size(), families.length);
       for (FileStatus f : families) {
         String familyStr = f.getPath().getName();
@@ -1185,8 +1186,9 @@ public class TestHFileOutputFormat2  {
         assertEquals("Incorrect bloom filter used for column family " + familyStr +
           "(reader: " + reader + ")",
           hcd.getBloomFilterType(), BloomType.valueOf(Bytes.toString(bloomFilter)));
-        assertEquals("Incorrect compression used for column family " + familyStr +
-          "(reader: " + reader + ")", hcd.getCompressionType(), reader.getFileContext().getCompression());
+        assertEquals(
+          "Incorrect compression used for column family " + familyStr + "(reader: " + reader + ")",
+          hcd.getCompressionType(), reader.getFileContext().getCompression());
       }
     } finally {
       dir.getFileSystem(conf).delete(dir, true);
@@ -1243,7 +1245,7 @@ public class TestHFileOutputFormat2  {
 
       // deep inspection: get the StoreFile dir
       final Path storePath = new Path(
-        FSUtils.getTableDir(FSUtils.getRootDir(conf), TABLE_NAMES[0]),
+        CommonFSUtils.getTableDir(CommonFSUtils.getRootDir(conf), TABLE_NAMES[0]),
           new Path(admin.getTableRegions(TABLE_NAMES[0]).get(0).getEncodedName(),
             Bytes.toString(FAMILIES[0])));
       assertEquals(0, fs.listStatus(storePath).length);
@@ -1324,7 +1326,7 @@ public class TestHFileOutputFormat2  {
 
       // deep inspection: get the StoreFile dir
       final Path storePath = new Path(
-        FSUtils.getTableDir(FSUtils.getRootDir(conf), TABLE_NAMES[0]),
+        CommonFSUtils.getTableDir(CommonFSUtils.getRootDir(conf), TABLE_NAMES[0]),
           new Path(admin.getTableRegions(TABLE_NAMES[0]).get(0).getEncodedName(),
             Bytes.toString(FAMILIES[0])));
       assertEquals(0, fs.listStatus(storePath).length);
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableSnapshotInputFormat.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableSnapshotInputFormat.java
index 1d5a5e0..89e7b49 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableSnapshotInputFormat.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableSnapshotInputFormat.java
@@ -31,7 +31,7 @@ import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.VerySlowMapReduceTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.mapreduce.Job;
 import org.junit.Before;
 import org.junit.BeforeClass;
@@ -53,17 +53,15 @@ public class TestMultiTableSnapshotInputFormat extends MultiTableInputFormatTest
 
   @BeforeClass
   public static void setUpSnapshots() throws Exception {
-
     TEST_UTIL.enableDebug(MultiTableSnapshotInputFormat.class);
     TEST_UTIL.enableDebug(MultiTableSnapshotInputFormatImpl.class);
 
     // take a snapshot of every table we have.
     for (String tableName : TABLES) {
-      SnapshotTestingUtils
-          .createSnapshotAndValidate(TEST_UTIL.getAdmin(), TableName.valueOf(tableName),
-              ImmutableList.of(INPUT_FAMILY), null,
-              snapshotNameForTable(tableName), FSUtils.getRootDir(TEST_UTIL.getConfiguration()),
-              TEST_UTIL.getTestFileSystem(), true);
+      SnapshotTestingUtils.createSnapshotAndValidate(TEST_UTIL.getAdmin(),
+        TableName.valueOf(tableName), ImmutableList.of(INPUT_FAMILY), null,
+        snapshotNameForTable(tableName), CommonFSUtils.getRootDir(TEST_UTIL.getConfiguration()),
+        TEST_UTIL.getTestFileSystem(), true);
     }
   }
 
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 9fec1b5..49f6ed4 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
@@ -18,8 +18,8 @@
 package org.apache.hadoop.hbase.mapreduce;
 
 import static org.junit.Assert.assertEquals;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.eq;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.eq;
 import static org.mockito.Mockito.doNothing;
 import static org.mockito.Mockito.verify;
 
@@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.junit.Before;
 import org.junit.ClassRule;
 import org.junit.Test;
@@ -77,13 +77,13 @@ public class TestMultiTableSnapshotInputFormatImpl {
 
     this.conf = new Configuration();
     this.rootDir = new Path("file:///test-root-dir");
-    FSUtils.setRootDir(conf, rootDir);
+    CommonFSUtils.setRootDir(conf, rootDir);
     this.snapshotScans = ImmutableMap.<String, Collection<Scan>>of("snapshot1",
         ImmutableList.of(new Scan(Bytes.toBytes("1"), Bytes.toBytes("2"))), "snapshot2",
         ImmutableList.of(new Scan(Bytes.toBytes("3"), Bytes.toBytes("4")),
             new Scan(Bytes.toBytes("5"), Bytes.toBytes("6"))));
 
-    this.restoreDir = new Path(FSUtils.getRootDir(conf), "restore-dir");
+    this.restoreDir = new Path(CommonFSUtils.getRootDir(conf), "restore-dir");
 
   }
 
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableSnapshotInputFormat.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableSnapshotInputFormat.java
index 358af24..5f187c6 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableSnapshotInputFormat.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableSnapshotInputFormat.java
@@ -44,7 +44,7 @@ import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.VerySlowMapReduceTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.RegionSplitter;
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.mapreduce.InputSplit;
@@ -325,7 +325,7 @@ public class TestTableSnapshotInputFormat extends TableSnapshotInputFormatTestBa
       admin.split(tableName, Bytes.toBytes("eee"));
       TestTableSnapshotScanner.blockUntilSplitFinished(UTIL, tableName, 2);
 
-      Path rootDir = FSUtils.getRootDir(UTIL.getConfiguration());
+      Path rootDir = CommonFSUtils.getRootDir(UTIL.getConfiguration());
       FileSystem fs = rootDir.getFileSystem(UTIL.getConfiguration());
 
       SnapshotTestingUtils.createSnapshotAndValidate(admin, tableName, Arrays.asList(FAMILIES),
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALPlayer.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALPlayer.java
index 7ddcb44..cf0464b 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALPlayer.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALPlayer.java
@@ -49,7 +49,7 @@ import org.apache.hadoop.hbase.mapreduce.WALPlayer.WALKeyValueMapper;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MapReduceTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.LauncherSecurityManager;
 import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.wal.WALEdit;
@@ -94,8 +94,8 @@ public class TestWALPlayer {
     conf= TEST_UTIL.getConfiguration();
     rootDir = TEST_UTIL.createRootDir();
     walRootDir = TEST_UTIL.createWALRootDir();
-    fs = FSUtils.getRootDirFileSystem(conf);
-    logFs = FSUtils.getWALFileSystem(conf);
+    fs = CommonFSUtils.getRootDirFileSystem(conf);
+    logFs = CommonFSUtils.getWALFileSystem(conf);
     cluster = TEST_UTIL.startMiniCluster();
   }
 
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALRecordReader.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALRecordReader.java
index 61ec12e..98e99db 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALRecordReader.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALRecordReader.java
@@ -41,7 +41,7 @@ import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
 import org.apache.hadoop.hbase.testclassification.MapReduceTests;
 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.CommonFSUtils;
 import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALFactory;
@@ -110,7 +110,7 @@ public class TestWALRecordReader {
 
     hbaseDir = TEST_UTIL.createRootDir();
     walRootDir = TEST_UTIL.createWALRootDir();
-    walFs = FSUtils.getWALFileSystem(conf);
+    walFs = CommonFSUtils.getWALFileSystem(conf);
     logDir = new Path(walRootDir, HConstants.HREGION_LOGDIR_NAME);
   }
 
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionTool.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionTool.java
index 9e7340f..26cee49 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionTool.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionTool.java
@@ -31,7 +31,7 @@ import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.util.ToolRunner;
 import org.junit.After;
 import org.junit.Before;
@@ -74,7 +74,7 @@ public class TestCompactionTool {
     }
     HStore store = region.getStore(HBaseTestingUtility.fam1);
     assertEquals(10, store.getStorefilesCount());
-    Path tableDir = FSUtils.getTableDir(rootDir, region.getRegionInfo().getTable());
+    Path tableDir = CommonFSUtils.getTableDir(rootDir, region.getRegionInfo().getTable());
     FileSystem fs = store.getFileSystem();
     String storePath = tableDir + "/" + region.getRegionInfo().getEncodedName() + "/"
       + Bytes.toString(HBaseTestingUtility.fam1);
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestVerifyReplication.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestVerifyReplication.java
index 50d8dff..6869de9 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestVerifyReplication.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestVerifyReplication.java
@@ -55,7 +55,7 @@ import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.mapreduce.Job;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -281,14 +281,14 @@ public class TestVerifyReplication extends TestReplicationBase {
     runSmallBatchTest();
 
     // Take source and target tables snapshot
-    Path rootDir = FSUtils.getRootDir(CONF1);
+    Path rootDir = CommonFSUtils.getRootDir(CONF1);
     FileSystem fs = rootDir.getFileSystem(CONF1);
     String sourceSnapshotName = "sourceSnapshot-" + System.currentTimeMillis();
     SnapshotTestingUtils.createSnapshotAndValidate(UTIL1.getAdmin(), tableName,
       Bytes.toString(famName), sourceSnapshotName, rootDir, fs, true);
 
     // Take target snapshot
-    Path peerRootDir = FSUtils.getRootDir(CONF2);
+    Path peerRootDir = CommonFSUtils.getRootDir(CONF2);
     FileSystem peerFs = peerRootDir.getFileSystem(CONF2);
     String peerSnapshotName = "peerSnapshot-" + System.currentTimeMillis();
     SnapshotTestingUtils.createSnapshotAndValidate(UTIL2.getAdmin(), tableName,
@@ -301,7 +301,7 @@ public class TestVerifyReplication extends TestReplicationBase {
     String[] args = new String[] { "--sourceSnapshotName=" + sourceSnapshotName,
       "--sourceSnapshotTmpDir=" + tmpPath1, "--peerSnapshotName=" + peerSnapshotName,
       "--peerSnapshotTmpDir=" + tmpPath2, "--peerFSAddress=" + peerFSAddress,
-      "--peerHBaseRootAddress=" + FSUtils.getRootDir(CONF2), UTIL2.getClusterKey(),
+      "--peerHBaseRootAddress=" + CommonFSUtils.getRootDir(CONF2), UTIL2.getClusterKey(),
       tableName.getNameAsString() };
     runVerifyReplication(args, NB_ROWS_IN_BATCH, 0);
     checkRestoreTmpDir(CONF1, tmpPath1, 1);
@@ -331,7 +331,7 @@ public class TestVerifyReplication extends TestReplicationBase {
     args = new String[] { "--sourceSnapshotName=" + sourceSnapshotName,
       "--sourceSnapshotTmpDir=" + tmpPath1, "--peerSnapshotName=" + peerSnapshotName,
       "--peerSnapshotTmpDir=" + tmpPath2, "--peerFSAddress=" + peerFSAddress,
-      "--peerHBaseRootAddress=" + FSUtils.getRootDir(CONF2), UTIL2.getClusterKey(),
+      "--peerHBaseRootAddress=" + CommonFSUtils.getRootDir(CONF2), UTIL2.getClusterKey(),
       tableName.getNameAsString() };
     runVerifyReplication(args, 0, NB_ROWS_IN_BATCH);
     checkRestoreTmpDir(CONF1, tmpPath1, 2);
@@ -386,14 +386,14 @@ public class TestVerifyReplication extends TestReplicationBase {
     runBatchCopyTest();
 
     // Take source and target tables snapshot
-    Path rootDir = FSUtils.getRootDir(CONF1);
+    Path rootDir = CommonFSUtils.getRootDir(CONF1);
     FileSystem fs = rootDir.getFileSystem(CONF1);
     String sourceSnapshotName = "sourceSnapshot-" + System.currentTimeMillis();
     SnapshotTestingUtils.createSnapshotAndValidate(UTIL1.getAdmin(), tableName,
             Bytes.toString(noRepfamName), sourceSnapshotName, rootDir, fs, true);
 
     // Take target snapshot
-    Path peerRootDir = FSUtils.getRootDir(CONF2);
+    Path peerRootDir = CommonFSUtils.getRootDir(CONF2);
     FileSystem peerFs = peerRootDir.getFileSystem(CONF2);
     String peerSnapshotName = "peerSnapshot-" + System.currentTimeMillis();
     SnapshotTestingUtils.createSnapshotAndValidate(UTIL2.getAdmin(), peerTableName,
@@ -407,7 +407,7 @@ public class TestVerifyReplication extends TestReplicationBase {
       "--sourceSnapshotName=" + sourceSnapshotName,
       "--sourceSnapshotTmpDir=" + tmpPath1, "--peerSnapshotName=" + peerSnapshotName,
       "--peerSnapshotTmpDir=" + tmpPath2, "--peerFSAddress=" + peerFSAddress,
-      "--peerHBaseRootAddress=" + FSUtils.getRootDir(CONF2), UTIL2.getClusterKey(),
+      "--peerHBaseRootAddress=" + CommonFSUtils.getRootDir(CONF2), UTIL2.getClusterKey(),
       tableName.getNameAsString() };
     runVerifyReplication(args, NB_ROWS_IN_BATCH, 0);
     checkRestoreTmpDir(CONF1, tmpPath1, 1);
@@ -438,7 +438,7 @@ public class TestVerifyReplication extends TestReplicationBase {
       "--sourceSnapshotName=" + sourceSnapshotName,
       "--sourceSnapshotTmpDir=" + tmpPath1, "--peerSnapshotName=" + peerSnapshotName,
       "--peerSnapshotTmpDir=" + tmpPath2, "--peerFSAddress=" + peerFSAddress,
-      "--peerHBaseRootAddress=" + FSUtils.getRootDir(CONF2), UTIL2.getClusterKey(),
+      "--peerHBaseRootAddress=" + CommonFSUtils.getRootDir(CONF2), UTIL2.getClusterKey(),
       tableName.getNameAsString() };
     runVerifyReplication(args, 0, NB_ROWS_IN_BATCH);
     checkRestoreTmpDir(CONF1, tmpPath1, 2);
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestVerifyReplicationAdjunct.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestVerifyReplicationAdjunct.java
index b3f4f6a..191b76b 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestVerifyReplicationAdjunct.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestVerifyReplicationAdjunct.java
@@ -21,6 +21,7 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
+
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
@@ -45,7 +46,7 @@ import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
@@ -56,6 +57,7 @@ import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 
 /**
@@ -282,14 +284,14 @@ public class TestVerifyReplicationAdjunct extends TestReplicationBase {
     runSmallBatchTest();
 
     // Take source and target tables snapshot
-    Path rootDir = FSUtils.getRootDir(CONF1);
+    Path rootDir = CommonFSUtils.getRootDir(CONF1);
     FileSystem fs = rootDir.getFileSystem(CONF1);
     String sourceSnapshotName = "sourceSnapshot-" + System.currentTimeMillis();
     SnapshotTestingUtils.createSnapshotAndValidate(UTIL1.getAdmin(), tableName,
       Bytes.toString(famName), sourceSnapshotName, rootDir, fs, true);
 
     // Take target snapshot
-    Path peerRootDir = FSUtils.getRootDir(CONF2);
+    Path peerRootDir = CommonFSUtils.getRootDir(CONF2);
     FileSystem peerFs = peerRootDir.getFileSystem(CONF2);
     String peerSnapshotName = "peerSnapshot-" + System.currentTimeMillis();
     SnapshotTestingUtils.createSnapshotAndValidate(UTIL2.getAdmin(), tableName,
@@ -302,7 +304,8 @@ public class TestVerifyReplicationAdjunct extends TestReplicationBase {
     String[] args = new String[] { "--sourceSnapshotName=" + sourceSnapshotName,
       "--sourceSnapshotTmpDir=" + temPath1, "--peerSnapshotName=" + peerSnapshotName,
       "--peerSnapshotTmpDir=" + temPath2, "--peerFSAddress=" + peerFSAddress,
-      "--peerHBaseRootAddress=" + FSUtils.getRootDir(CONF2), "2", tableName.getNameAsString() };
+      "--peerHBaseRootAddress=" + CommonFSUtils.getRootDir(CONF2), "2",
+      tableName.getNameAsString() };
     TestVerifyReplication.runVerifyReplication(args, NB_ROWS_IN_BATCH, 0);
     TestVerifyReplication.checkRestoreTmpDir(CONF1, temPath1, 1);
     TestVerifyReplication.checkRestoreTmpDir(CONF2, temPath2, 1);
@@ -331,7 +334,8 @@ public class TestVerifyReplicationAdjunct extends TestReplicationBase {
     args = new String[] { "--sourceSnapshotName=" + sourceSnapshotName,
       "--sourceSnapshotTmpDir=" + temPath1, "--peerSnapshotName=" + peerSnapshotName,
       "--peerSnapshotTmpDir=" + temPath2, "--peerFSAddress=" + peerFSAddress,
-      "--peerHBaseRootAddress=" + FSUtils.getRootDir(CONF2), "2", tableName.getNameAsString() };
+      "--peerHBaseRootAddress=" + CommonFSUtils.getRootDir(CONF2), "2",
+      tableName.getNameAsString() };
     TestVerifyReplication.runVerifyReplication(args, 0, NB_ROWS_IN_BATCH);
     TestVerifyReplication.checkRestoreTmpDir(CONF1, temPath1, 2);
     TestVerifyReplication.checkRestoreTmpDir(CONF2, temPath2, 2);
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestVerifyReplicationCrossDiffHdfs.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestVerifyReplicationCrossDiffHdfs.java
index a07e0a8..639f686 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestVerifyReplicationCrossDiffHdfs.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestVerifyReplicationCrossDiffHdfs.java
@@ -22,7 +22,6 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -46,7 +45,6 @@ import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.CommonFSUtils;
-import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
 import org.apache.hadoop.mapreduce.Job;
 import org.junit.AfterClass;
@@ -165,14 +163,14 @@ public class TestVerifyReplicationCrossDiffHdfs {
 
   @Test
   public void testVerifyRepBySnapshot() throws Exception {
-    Path rootDir = FSUtils.getRootDir(conf1);
+    Path rootDir = CommonFSUtils.getRootDir(conf1);
     FileSystem fs = rootDir.getFileSystem(conf1);
     String sourceSnapshotName = "sourceSnapshot-" + System.currentTimeMillis();
     SnapshotTestingUtils.createSnapshotAndValidate(util1.getAdmin(), TABLE_NAME, new String(FAMILY),
       sourceSnapshotName, rootDir, fs, true);
 
     // Take target snapshot
-    Path peerRootDir = FSUtils.getRootDir(conf2);
+    Path peerRootDir = CommonFSUtils.getRootDir(conf2);
     FileSystem peerFs = peerRootDir.getFileSystem(conf2);
     String peerSnapshotName = "peerSnapshot-" + System.currentTimeMillis();
     SnapshotTestingUtils.createSnapshotAndValidate(util2.getAdmin(), TABLE_NAME, new String(FAMILY),
@@ -185,7 +183,7 @@ public class TestVerifyReplicationCrossDiffHdfs {
     String[] args = new String[] { "--sourceSnapshotName=" + sourceSnapshotName,
       "--sourceSnapshotTmpDir=" + temPath1, "--peerSnapshotName=" + peerSnapshotName,
       "--peerSnapshotTmpDir=" + temPath2, "--peerFSAddress=" + peerFSAddress,
-      "--peerHBaseRootAddress=" + FSUtils.getRootDir(conf2), PEER_ID, TABLE_NAME.toString() };
+      "--peerHBaseRootAddress=" + CommonFSUtils.getRootDir(conf2), PEER_ID, TABLE_NAME.toString() };
 
     // Use the yarn's config override the source cluster's config.
     Configuration newConf = HBaseConfiguration.create(conf1);
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 8d65ac1..76d337f 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
@@ -21,6 +21,7 @@ import static org.apache.hadoop.util.ToolRunner.run;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
+
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashSet;
@@ -41,7 +42,7 @@ import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.VerySlowMapReduceTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -53,6 +54,7 @@ import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
 
@@ -285,7 +287,7 @@ public class TestExportSnapshot {
           snapshotFiles.add(hfile);
           if (!storeFile.hasReference()) {
             verifyNonEmptyFile(new Path(exportedArchive,
-              new Path(FSUtils.getTableDir(new Path("./"), tableName),
+              new Path(CommonFSUtils.getTableDir(new Path("./"), tableName),
                   new Path(regionInfo.getEncodedName(), new Path(family, hfile)))));
           }
         }
@@ -308,7 +310,7 @@ public class TestExportSnapshot {
     Set<String> files = new HashSet<>();
     LOG.debug("List files in {} in root {} at {}", fs, root, dir);
     int rootPrefix = root.makeQualified(fs.getUri(), fs.getWorkingDirectory()).toString().length();
-    FileStatus[] list = FSUtils.listStatus(fs, dir);
+    FileStatus[] list = CommonFSUtils.listStatus(fs, dir);
     if (list != null) {
       for (FileStatus fstat: list) {
         LOG.debug(Objects.toString(fstat.getPath()));
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 d7c52ec..e247e75 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
@@ -57,7 +57,7 @@ org.apache.hadoop.hbase.security.visibility.VisibilityConstants;
 org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
 org.apache.hadoop.hbase.tool.CanaryTool;
 org.apache.hadoop.hbase.util.Bytes;
-org.apache.hadoop.hbase.util.FSUtils;
+org.apache.hadoop.hbase.util.CommonFSUtils;
 org.apache.hadoop.hbase.util.JvmVersion;
 org.apache.hadoop.util.StringUtils;
 </%import>
@@ -354,7 +354,7 @@ AssignmentManager assignmentManager = master.getAssignmentManager();
                 </tr>
                 <tr>
                     <td>HBase Root Directory</td>
-                    <td><% FSUtils.getRootDir(master.getConfiguration()).toString() %></td>
+                    <td><% CommonFSUtils.getRootDir(master.getConfiguration()).toString() %></td>
                     <td>Location of HBase home directory</td>
                 </tr>
                 <tr>
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HFileArchiver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HFileArchiver.java
index ce5cd0e..8adc9b5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HFileArchiver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HFileArchiver.java
@@ -86,7 +86,7 @@ public class HFileArchiver {
    */
   public static boolean exists(Configuration conf, FileSystem fs, RegionInfo info)
       throws IOException {
-    Path rootDir = FSUtils.getRootDir(conf);
+    Path rootDir = CommonFSUtils.getRootDir(conf);
     Path regionDir = FSUtils.getRegionDirFromRootDir(rootDir, info);
     return fs.exists(regionDir);
   }
@@ -99,8 +99,8 @@ public class HFileArchiver {
    */
   public static void archiveRegion(Configuration conf, FileSystem fs, RegionInfo info)
       throws IOException {
-    Path rootDir = FSUtils.getRootDir(conf);
-    archiveRegion(fs, rootDir, FSUtils.getTableDir(rootDir, info.getTable()),
+    Path rootDir = CommonFSUtils.getRootDir(conf);
+    archiveRegion(fs, rootDir, CommonFSUtils.getTableDir(rootDir, info.getTable()),
       FSUtils.getRegionDirFromRootDir(rootDir, info));
   }
 
@@ -135,8 +135,7 @@ public class HFileArchiver {
     // make sure the regiondir lives under the tabledir
     Preconditions.checkArgument(regionDir.toString().startsWith(tableDir.toString()));
     Path regionArchiveDir = HFileArchiveUtil.getRegionArchiveDir(rootdir,
-        FSUtils.getTableName(tableDir),
-        regionDir.getName());
+      CommonFSUtils.getTableName(tableDir), regionDir.getName());
 
     FileStatusConverter getAsFile = new FileStatusConverter(fs);
     // otherwise, we attempt to archive the store files
@@ -150,7 +149,7 @@ public class HFileArchiver {
         return dirFilter.accept(file) && !file.getName().startsWith(".");
       }
     };
-    FileStatus[] storeDirs = FSUtils.listStatus(fs, regionDir, nonHidden);
+    FileStatus[] storeDirs = CommonFSUtils.listStatus(fs, regionDir, nonHidden);
     // if there no files, we can just delete the directory and return;
     if (storeDirs == null) {
       LOG.debug("Directory {} empty.", regionDir);
@@ -263,7 +262,7 @@ public class HFileArchiver {
    */
   public static void archiveFamilyByFamilyDir(FileSystem fs, Configuration conf,
       RegionInfo parent, Path familyDir, byte[] family) throws IOException {
-    FileStatus[] storeFiles = FSUtils.listStatus(fs, familyDir);
+    FileStatus[] storeFiles = CommonFSUtils.listStatus(fs, familyDir);
     if (storeFiles == null) {
       LOG.debug("No files to dispose of in {}, family={}", parent.getRegionNameAsString(),
           Bytes.toString(family));
@@ -694,7 +693,7 @@ public class HFileArchiver {
     public boolean moveAndClose(Path dest) throws IOException {
       this.close();
       Path p = this.getPath();
-      return FSUtils.renameAndSetModifyTime(fs, p, dest);
+      return CommonFSUtils.renameAndSetModifyTime(fs, p, dest);
     }
 
     /**
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/LongTermArchivingHFileCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/LongTermArchivingHFileCleaner.java
index 9212063..946f759 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/LongTermArchivingHFileCleaner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/LongTermArchivingHFileCleaner.java
@@ -18,14 +18,13 @@
 package org.apache.hadoop.hbase.backup.example;
 
 import java.io.IOException;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.master.cleaner.BaseHFileCleanerDelegate;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
@@ -59,7 +58,7 @@ public class LongTermArchivingHFileCleaner extends BaseHFileCleanerDelegate {
       
       Path file = fStat.getPath();
       // check to see if
-      FileStatus[] deleteStatus = FSUtils.listStatus(this.fs, file, null);
+      FileStatus[] deleteStatus = CommonFSUtils.listStatus(this.fs, file, null);
       // if the file doesn't exist, then it can be deleted (but should never
       // happen since deleted files shouldn't get passed in)
       if (deleteStatus == null) {
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 a23cede..1e2b699 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
@@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.client;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -31,7 +30,7 @@ import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
 import org.apache.hadoop.hbase.mob.MobFileCache;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -58,8 +57,8 @@ public class ClientSideRegionScanner extends AbstractClientScanner {
     htd = TableDescriptorBuilder.newBuilder(htd).setReadOnly(true).build();
 
     // open region from the snapshot directory
-    region = HRegion.newHRegion(FSUtils.getTableDir(rootDir, htd.getTableName()), null, fs, conf,
-      hri, htd, null);
+    region = HRegion.newHRegion(CommonFSUtils.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
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/TableSnapshotScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/TableSnapshotScanner.java
index 193b391..295a2c7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/TableSnapshotScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/TableSnapshotScanner.java
@@ -22,7 +22,6 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.UUID;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -31,7 +30,7 @@ import org.apache.hadoop.hbase.PrivateCellUtil;
 import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper;
 import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
 import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -96,7 +95,7 @@ public class TableSnapshotScanner extends AbstractClientScanner {
    */
   public TableSnapshotScanner(Configuration conf, Path restoreDir, String snapshotName, Scan scan)
       throws IOException {
-    this(conf, FSUtils.getRootDir(conf), restoreDir, snapshotName, scan);
+    this(conf, CommonFSUtils.getRootDir(conf), restoreDir, snapshotName, scan);
   }
 
   public TableSnapshotScanner(Configuration conf, Path rootDir, Path restoreDir,
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/HFileSystem.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/HFileSystem.java
index bc3d85e..2be48b4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/HFileSystem.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/HFileSystem.java
@@ -38,7 +38,7 @@ import org.apache.hadoop.fs.FilterFileSystem;
 import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.ReflectionUtils;
 import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
 import org.apache.hadoop.hdfs.DFSClient;
@@ -154,7 +154,7 @@ public class HFileSystem extends FilterFileSystem {
    * 'COLD', 'WARM', 'HOT', 'ONE_SSD', 'ALL_SSD', 'LAZY_PERSIST'.
    */
   public void setStoragePolicy(Path path, String policyName) {
-    FSUtils.setStoragePolicy(this.fs, path, policyName);
+    CommonFSUtils.setStoragePolicy(this.fs, path, policyName);
   }
 
   /**
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/FileLink.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/FileLink.java
index 184f788..1e7e93b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/FileLink.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/FileLink.java
@@ -18,24 +18,23 @@
 
 package org.apache.hadoop.hbase.io;
 
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.FileNotFoundException;
 import java.util.List;
-
 import org.apache.hadoop.fs.CanSetDropBehind;
 import org.apache.hadoop.fs.CanSetReadahead;
 import org.apache.hadoop.fs.CanUnbuffer;
 import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PositionedReadable;
 import org.apache.hadoop.fs.Seekable;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -115,7 +114,7 @@ public class FileLink {
 
     public FileLinkInputStream(final FileSystem fs, final FileLink fileLink)
         throws IOException {
-      this(fs, fileLink, FSUtils.getDefaultBufferSize(fs));
+      this(fs, fileLink, CommonFSUtils.getDefaultBufferSize(fs));
     }
 
     public FileLinkInputStream(final FileSystem fs, final FileLink fileLink, int bufferSize)
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HFileLink.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HFileLink.java
index 2dc4aeb..cc723ee 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HFileLink.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HFileLink.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.io;
 import java.io.IOException;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -32,7 +31,7 @@ import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.mob.MobConstants;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.HFileArchiveUtil;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -120,7 +119,7 @@ public class HFileLink extends FileLink {
    */
   public static final HFileLink buildFromHFileLinkPattern(Configuration conf, Path hFileLinkPattern)
           throws IOException {
-    return buildFromHFileLinkPattern(FSUtils.getRootDir(conf),
+    return buildFromHFileLinkPattern(CommonFSUtils.getRootDir(conf),
             HFileArchiveUtil.getArchivePath(conf), hFileLinkPattern);
   }
 
@@ -235,7 +234,7 @@ public class HFileLink extends FileLink {
     String regionName = m.group(3);
     String hfileName = m.group(4);
     String familyName = path.getParent().getName();
-    Path tableDir = FSUtils.getTableDir(new Path("./"), tableName);
+    Path tableDir = CommonFSUtils.getTableDir(new Path("./"), tableName);
     return new Path(tableDir, new Path(regionName, new Path(familyName,
         hfileName)));
   }
@@ -396,7 +395,7 @@ public class HFileLink extends FileLink {
       final String hfileName, final boolean createBackRef) throws IOException {
     String familyName = dstFamilyPath.getName();
     String regionName = dstFamilyPath.getParent().getName();
-    String tableName = FSUtils.getTableName(dstFamilyPath.getParent().getParent())
+    String tableName = CommonFSUtils.getTableName(dstFamilyPath.getParent().getParent())
         .getNameAsString();
 
     String name = createHFileLinkName(linkedTable, linkedRegion, hfileName);
@@ -501,9 +500,9 @@ public class HFileLink extends FileLink {
     Path regionPath = familyPath.getParent();
     Path tablePath = regionPath.getParent();
 
-    String linkName = createHFileLinkName(FSUtils.getTableName(tablePath),
+    String linkName = createHFileLinkName(CommonFSUtils.getTableName(tablePath),
             regionPath.getName(), hfileName);
-    Path linkTableDir = FSUtils.getTableDir(rootDir, linkTableName);
+    Path linkTableDir = CommonFSUtils.getTableDir(rootDir, linkTableName);
     Path regionDir = HRegion.getRegionDir(linkTableDir, linkRegionName);
     return new Path(new Path(regionDir, familyPath.getName()), linkName);
   }
@@ -527,7 +526,7 @@ public class HFileLink extends FileLink {
    */
   public static Path getHFileFromBackReference(final Configuration conf, final Path linkRefPath)
       throws IOException {
-    return getHFileFromBackReference(FSUtils.getRootDir(conf), linkRefPath);
+    return getHFileFromBackReference(CommonFSUtils.getRootDir(conf), linkRefPath);
   }
 
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutput.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutput.java
index ed9da5f..2495846 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutput.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutput.java
@@ -103,8 +103,6 @@ public class FanOutOneBlockAsyncDFSOutput implements AsyncFSOutput {
 
   private final Configuration conf;
 
-  private final FSUtils fsUtils;
-
   private final DistributedFileSystem dfs;
 
   private final DFSClient client;
@@ -325,12 +323,11 @@ public class FanOutOneBlockAsyncDFSOutput implements AsyncFSOutput {
     }
   }
 
-  FanOutOneBlockAsyncDFSOutput(Configuration conf, FSUtils fsUtils, DistributedFileSystem dfs,
+  FanOutOneBlockAsyncDFSOutput(Configuration conf,DistributedFileSystem dfs,
       DFSClient client, ClientProtocol namenode, String clientName, String src, long fileId,
       LocatedBlock locatedBlock, Encryptor encryptor, List<Channel> datanodeList,
       DataChecksum summer, ByteBufAllocator alloc) {
     this.conf = conf;
-    this.fsUtils = fsUtils;
     this.dfs = dfs;
     this.client = client;
     this.namenode = namenode;
@@ -555,7 +552,7 @@ public class FanOutOneBlockAsyncDFSOutput implements AsyncFSOutput {
     datanodeList.forEach(ch -> ch.close());
     datanodeList.forEach(ch -> ch.closeFuture().awaitUninterruptibly());
     endFileLease(client, fileId);
-    fsUtils.recoverFileLease(dfs, new Path(src), conf,
+    FSUtils.recoverFileLease(dfs, new Path(src), conf,
       reporter == null ? new CancelOnClose(client) : reporter);
   }
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java
index e372726..2978eb9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java
@@ -48,7 +48,6 @@ import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hbase.client.ConnectionUtils;
 import org.apache.hadoop.hbase.util.CancelableProgressable;
-import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSOutputStream;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
@@ -514,7 +513,6 @@ public final class FanOutOneBlockAsyncDFSOutputHelper {
       boolean overwrite, boolean createParent, short replication, long blockSize,
       EventLoopGroup eventLoopGroup, Class<? extends Channel> channelClass) throws IOException {
     Configuration conf = dfs.getConf();
-    FSUtils fsUtils = FSUtils.getInstance(dfs, conf);
     DFSClient client = dfs.getClient();
     String clientName = client.getClientName();
     ClientProtocol namenode = client.getNamenode();
@@ -557,8 +555,8 @@ public final class FanOutOneBlockAsyncDFSOutputHelper {
         }
         Encryptor encryptor = createEncryptor(conf, stat, client);
         FanOutOneBlockAsyncDFSOutput output =
-          new FanOutOneBlockAsyncDFSOutput(conf, fsUtils, dfs, client, namenode, clientName, src,
-              stat.getFileId(), locatedBlock, encryptor, datanodeList, summer, ALLOC);
+          new FanOutOneBlockAsyncDFSOutput(conf, dfs, client, namenode, clientName, src,
+            stat.getFileId(), locatedBlock, encryptor, datanodeList, summer, ALLOC);
         succ = true;
         return output;
       } catch (RemoteException e) {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java
index 88f73f7..57db206 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java
@@ -30,7 +30,6 @@ import com.codahale.metrics.MetricRegistry;
 import com.codahale.metrics.ScheduledReporter;
 import com.codahale.metrics.Snapshot;
 import com.codahale.metrics.Timer;
-
 import java.io.ByteArrayOutputStream;
 import java.io.DataInput;
 import java.io.IOException;
@@ -47,7 +46,6 @@ import java.util.Set;
 import java.util.SortedMap;
 import java.util.TimeZone;
 import java.util.concurrent.TimeUnit;
-
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
@@ -73,11 +71,10 @@ import org.apache.hadoop.hbase.util.BloomFilter;
 import org.apache.hadoop.hbase.util.BloomFilterFactory;
 import org.apache.hadoop.hbase.util.BloomFilterUtil;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.HFileArchiveUtil;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
-
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.yetus.audience.InterfaceStability;
 import org.slf4j.Logger;
@@ -210,8 +207,8 @@ public class HFilePrettyPrinter extends Configured implements Tool {
       String regionName = cmd.getOptionValue("r");
       byte[] rn = Bytes.toBytes(regionName);
       byte[][] hri = HRegionInfo.parseRegionName(rn);
-      Path rootDir = FSUtils.getRootDir(getConf());
-      Path tableDir = FSUtils.getTableDir(rootDir, TableName.valueOf(hri[0]));
+      Path rootDir = CommonFSUtils.getRootDir(getConf());
+      Path tableDir = CommonFSUtils.getTableDir(rootDir, TableName.valueOf(hri[0]));
       String enc = HRegionInfo.encodeRegionName(rn);
       Path regionDir = new Path(tableDir, enc);
       if (verbose)
@@ -253,9 +250,10 @@ public class HFilePrettyPrinter extends Configured implements Tool {
       throw new RuntimeException("A Configuration instance must be provided.");
     }
     try {
-      FSUtils.setFsDefault(getConf(), FSUtils.getRootDir(getConf()));
-      if (!parseOptions(args))
+      CommonFSUtils.setFsDefault(getConf(), CommonFSUtils.getRootDir(getConf()));
+      if (!parseOptions(args)) {
         return 1;
+      }
     } catch (IOException ex) {
       LOG.error("Error parsing command-line options", ex);
       return 1;
@@ -291,7 +289,7 @@ public class HFilePrettyPrinter extends Configured implements Tool {
     }
 
     if (checkRootDir) {
-      Path rootPath = FSUtils.getRootDir(getConf());
+      Path rootPath = CommonFSUtils.getRootDir(getConf());
       String rootString = rootPath + rootPath.SEPARATOR;
       if (!file.toString().startsWith(rootString)) {
         // First we see if fully-qualified URI matches the root dir. It might
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java
index 604ac1f..dd05963 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java
@@ -47,11 +47,13 @@ import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.BloomFilterWriter;
 import org.apache.hadoop.hbase.util.ByteBufferUtils;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.io.Writable;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 
 /**
@@ -287,7 +289,7 @@ public class HFileWriterImpl implements HFile.Writer {
   /** A helper method to create HFile output streams in constructors */
   protected static FSDataOutputStream createOutputStream(Configuration conf,
       FileSystem fs, Path path, InetSocketAddress[] favoredNodes) throws IOException {
-    FsPermission perms = FSUtils.getFilePermissions(fs, conf,
+    FsPermission perms = CommonFSUtils.getFilePermissions(fs, conf,
         HConstants.DATA_FILE_UMASK_KEY);
     return FSUtils.create(conf, fs, path, perms, favoredNodes);
   }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CachedClusterId.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CachedClusterId.java
index 4bc444b..2e7baae 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CachedClusterId.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CachedClusterId.java
@@ -26,10 +26,12 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.ClusterId;
 import org.apache.hadoop.hbase.Server;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.yetus.audience.InterfaceAudience;
 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;
 
@@ -65,7 +67,7 @@ public class CachedClusterId {
   private AtomicInteger cacheMisses = new AtomicInteger(0);
 
   public CachedClusterId(Server server, Configuration conf) throws IOException {
-    this.rootDir = FSUtils.getRootDir(conf);
+    this.rootDir = CommonFSUtils.getRootDir(conf);
     this.fs = rootDir.getFileSystem(conf);
     this.server = server;
   }
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 4b792f9..0b42561 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
@@ -28,7 +28,6 @@ import java.util.Properties;
 import java.util.TreeMap;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.stream.Collectors;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -57,8 +56,8 @@ import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.PairOfSameType;
 import org.apache.hadoop.hbase.util.Threads;
@@ -248,7 +247,7 @@ public class CatalogJanitor extends ScheduledChore {
       throws IOException {
     FileSystem fs = this.services.getMasterFileSystem().getFileSystem();
     Path rootdir = this.services.getMasterFileSystem().getRootDir();
-    Path tabledir = FSUtils.getTableDir(rootdir, mergedRegion.getTable());
+    Path tabledir = CommonFSUtils.getTableDir(rootdir, mergedRegion.getTable());
     TableDescriptor htd = getDescriptor(mergedRegion.getTable());
     HRegionFileSystem regionFs = null;
     try {
@@ -373,14 +372,14 @@ public class CatalogJanitor extends ScheduledChore {
 
     FileSystem fs = this.services.getMasterFileSystem().getFileSystem();
     Path rootdir = this.services.getMasterFileSystem().getRootDir();
-    Path tabledir = FSUtils.getTableDir(rootdir, daughter.getTable());
+    Path tabledir = CommonFSUtils.getTableDir(rootdir, daughter.getTable());
 
     Path daughterRegionDir = new Path(tabledir, daughter.getEncodedName());
 
     HRegionFileSystem regionFs;
 
     try {
-      if (!FSUtils.isExists(fs, daughterRegionDir)) {
+      if (!CommonFSUtils.isExists(fs, daughterRegionDir)) {
         return new Pair<>(Boolean.FALSE, Boolean.FALSE);
       }
     } catch (IOException ioe) {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
index 6badf4e..72a5462 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hbase.master;
 
 import java.io.IOException;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -43,6 +42,7 @@ import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.security.access.SnapshotScannerHDFSAclHelper;
 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.ipc.RemoteException;
@@ -107,16 +107,16 @@ public class MasterFileSystem {
     // mismatched filesystems if hbase.rootdir is hdfs and fs.defaultFS is
     // default localfs.  Presumption is that rootdir is fully-qualified before
     // we get to here with appropriate fs scheme.
-    this.rootdir = FSUtils.getRootDir(conf);
+    this.rootdir = CommonFSUtils.getRootDir(conf);
     this.tempdir = new Path(this.rootdir, HConstants.HBASE_TEMP_DIRECTORY);
     // Cover both bases, the old way of setting default fs and the new.
     // We're supposed to run on 0.20 and 0.21 anyways.
     this.fs = this.rootdir.getFileSystem(conf);
-    this.walRootDir = FSUtils.getWALRootDir(conf);
-    this.walFs = FSUtils.getWALFileSystem(conf);
-    FSUtils.setFsDefault(conf, new Path(this.walFs.getUri()));
+    this.walRootDir = CommonFSUtils.getWALRootDir(conf);
+    this.walFs = CommonFSUtils.getWALFileSystem(conf);
+    CommonFSUtils.setFsDefault(conf, new Path(this.walFs.getUri()));
     walFs.setConf(conf);
-    FSUtils.setFsDefault(conf, new Path(this.fs.getUri()));
+    CommonFSUtils.setFsDefault(conf, new Path(this.fs.getUri()));
     // make sure the fs has the same conf
     fs.setConf(conf);
     this.secureRootSubDirPerms = new FsPermission(conf.get("hbase.rootdir.perms", "700"));
@@ -443,7 +443,7 @@ public class MasterFileSystem {
   public void deleteFamilyFromFS(Path rootDir, RegionInfo region, byte[] familyName)
       throws IOException {
     // archive family store files
-    Path tableDir = FSUtils.getTableDir(rootDir, region.getTable());
+    Path tableDir = CommonFSUtils.getTableDir(rootDir, region.getTable());
     HFileArchiver.archiveFamily(fs, conf, region, tableDir, familyName);
 
     // delete the family folder
@@ -463,6 +463,6 @@ public class MasterFileSystem {
   }
 
   public void logFileSystemState(Logger log) throws IOException {
-    FSUtils.logFileSystemState(fs, rootdir, log);
+    CommonFSUtils.logFileSystemState(fs, rootdir, log);
   }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterWalManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterWalManager.java
index fce6588..3f6bd8a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterWalManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterWalManager.java
@@ -44,6 +44,7 @@ import org.apache.hadoop.hbase.wal.WALSplitter;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 
 /**
@@ -188,7 +189,7 @@ public class MasterWalManager {
    */
   public FileStatus[] getWALDirPaths(final PathFilter filter) throws IOException {
     Path walDirPath = new Path(CommonFSUtils.getWALRootDir(conf), HConstants.HREGION_LOGDIR_NAME);
-    FileStatus[] walDirForServerNames = FSUtils.listStatus(fs, walDirPath, filter);
+    FileStatus[] walDirForServerNames = CommonFSUtils.listStatus(fs, walDirPath, filter);
     return walDirForServerNames == null? new FileStatus[0]: walDirForServerNames;
   }
 
@@ -215,7 +216,7 @@ public class MasterWalManager {
       }
       try {
         if (!this.fs.exists(logsDirPath)) return serverNames;
-        FileStatus[] logFolders = FSUtils.listStatus(this.fs, logsDirPath, null);
+        FileStatus[] logFolders = CommonFSUtils.listStatus(this.fs, logsDirPath, null);
         // Get online servers after getting log folders to avoid log folder deletion of newly
         // checked in region servers . see HBASE-5916
         Set<ServerName> onlineServers = services.getServerManager().getOnlineServers().keySet();
@@ -225,7 +226,7 @@ public class MasterWalManager {
           return serverNames;
         }
         for (FileStatus status : logFolders) {
-          FileStatus[] curLogFiles = FSUtils.listStatus(this.fs, status.getPath(), null);
+          FileStatus[] curLogFiles = CommonFSUtils.listStatus(this.fs, status.getPath(), null);
           if (curLogFiles == null || curLogFiles.length == 0) {
             // Empty log folder. No recovery needed
             continue;
@@ -372,13 +373,13 @@ public class MasterWalManager {
           AbstractFSWALProvider.getWALDirectoryName(serverName.toString()));
       Path splitDir = logDir.suffix(AbstractFSWALProvider.SPLITTING_EXT);
       if (fs.exists(splitDir)) {
-        FileStatus[] logfiles = FSUtils.listStatus(fs, splitDir, META_FILTER);
+        FileStatus[] logfiles = CommonFSUtils.listStatus(fs, splitDir, META_FILTER);
         if (logfiles != null) {
           for (FileStatus status : logfiles) {
             if (!status.isDir()) {
               Path newPath = AbstractFSWAL.getWALArchivePath(this.oldLogDir,
                   status.getPath());
-              if (!FSUtils.renameAndSetModifyTime(fs, status.getPath(), newPath)) {
+              if (!CommonFSUtils.renameAndSetModifyTime(fs, status.getPath(), newPath)) {
                 LOG.warn("Unable to move  " + status.getPath() + " to " + newPath);
               } else {
                 LOG.debug("Archived meta log " + status.getPath() + " to " + newPath);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
index e99bc16..3762817 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
@@ -35,7 +35,6 @@ import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.atomic.AtomicInteger;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -52,13 +51,14 @@ import org.apache.hadoop.hbase.log.HBaseMarkers;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
 import org.apache.hadoop.hbase.procedure2.util.StringUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.HasThread;
 import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 
 /**
@@ -173,7 +173,7 @@ public class SplitLogManager {
         LOG.warn(logDir + " doesn't exist. Nothing to do!");
         continue;
       }
-      FileStatus[] logfiles = FSUtils.listStatus(fs, logDir, filter);
+      FileStatus[] logfiles = CommonFSUtils.listStatus(fs, logDir, filter);
       if (logfiles == null || logfiles.length == 0) {
         LOG.info("{} dir is empty, no logs to split.", logDir);
       } else {
@@ -254,7 +254,7 @@ public class SplitLogManager {
         // recover-lease is done. totalSize will be under in most cases and the
         // metrics that it drives will also be under-reported.
         totalSize += lf.getLen();
-        String pathToLog = FSUtils.removeWALRootPath(lf.getPath(), conf);
+        String pathToLog = CommonFSUtils.removeWALRootPath(lf.getPath(), conf);
         if (!enqueueSplitTask(pathToLog, batch)) {
           throw new IOException("duplicate log split scheduled for " + lf.getPath());
         }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCRegionProcedure.java
index 8529a49..10aa0c1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCRegionProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCRegionProcedure.java
@@ -31,7 +31,7 @@ import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
 import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
 import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -90,14 +90,14 @@ public class GCRegionProcedure extends AbstractStateMachineRegionProcedure<GCReg
           }
           FileSystem walFs = mfs.getWALFileSystem();
           // Cleanup the directories on WAL filesystem also
-          Path regionWALDir = FSUtils.getWALRegionDir(env.getMasterConfiguration(),
+          Path regionWALDir = CommonFSUtils.getWALRegionDir(env.getMasterConfiguration(),
             getRegion().getTable(), getRegion().getEncodedName());
           if (walFs.exists(regionWALDir)) {
             if (!walFs.delete(regionWALDir, true)) {
               LOG.debug("Failed to delete {}", regionWALDir);
             }
           }
-          Path wrongRegionWALDir = FSUtils.getWrongWALRegionDir(env.getMasterConfiguration(),
+          Path wrongRegionWALDir = CommonFSUtils.getWrongWALRegionDir(env.getMasterConfiguration(),
             getRegion().getTable(), getRegion().getEncodedName());
           if (walFs.exists(wrongRegionWALDir)) {
             if (!walFs.delete(wrongRegionWALDir, true)) {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
index 7389cdb..0625db7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
@@ -24,7 +24,6 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 import java.util.stream.Stream;
-
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.MetaMutationAnnotation;
@@ -56,13 +55,14 @@ import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
 import org.apache.hadoop.hbase.regionserver.HStoreFile;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.wal.WALSplitUtil;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
@@ -575,7 +575,7 @@ public class MergeTableRegionsProcedure
    */
   private void createMergedRegion(final MasterProcedureEnv env) throws IOException {
     final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
-    final Path tabledir = FSUtils.getTableDir(mfs.getRootDir(), regionsToMerge[0].getTable());
+    final Path tabledir = CommonFSUtils.getTableDir(mfs.getRootDir(), regionsToMerge[0].getTable());
     final FileSystem fs = mfs.getFileSystem();
     HRegionFileSystem mergeRegionFs = null;
     for (RegionInfo ri: this.regionsToMerge) {
@@ -624,7 +624,7 @@ public class MergeTableRegionsProcedure
   private void cleanupMergedRegion(final MasterProcedureEnv env) throws IOException {
     final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
     TableName tn = this.regionsToMerge[0].getTable();
-    final Path tabledir = FSUtils.getTableDir(mfs.getRootDir(), tn);
+    final Path tabledir = CommonFSUtils.getTableDir(mfs.getRootDir(), tn);
     final FileSystem fs = mfs.getFileSystem();
     // See createMergedRegion above where we specify the merge dir as being in the
     // FIRST merge parent region.
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 0d62828..5f44fd1 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
@@ -33,7 +33,6 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.stream.Stream;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -65,6 +64,7 @@ import org.apache.hadoop.hbase.regionserver.HStoreFile;
 import org.apache.hadoop.hbase.regionserver.RegionSplitPolicy;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.Pair;
@@ -597,7 +597,7 @@ public class SplitTableRegionProcedure
   @VisibleForTesting
   public void createDaughterRegions(final MasterProcedureEnv env) throws IOException {
     final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
-    final Path tabledir = FSUtils.getTableDir(mfs.getRootDir(), getTableName());
+    final Path tabledir = CommonFSUtils.getTableDir(mfs.getRootDir(), getTableName());
     final FileSystem fs = mfs.getFileSystem();
     HRegionFileSystem regionFs = HRegionFileSystem.openRegionFromFileSystem(
       env.getMasterConfiguration(), fs, tabledir, getParentRegion(), false);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/HFileLinkCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/HFileLinkCleaner.java
index db5230c..a99c784 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/HFileLinkCleaner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/HFileLinkCleaner.java
@@ -18,19 +18,18 @@
 package org.apache.hadoop.hbase.master.cleaner;
 
 import java.io.IOException;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.io.HFileLink;
+import org.apache.hadoop.hbase.mob.MobUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hbase.io.HFileLink;
-import org.apache.hadoop.hbase.mob.MobUtils;
-import org.apache.hadoop.hbase.util.FSUtils;
 
 /**
  * HFileLink cleaner that determines if a hfile should be deleted.
@@ -62,7 +61,7 @@ public class HFileLinkCleaner extends BaseHFileCleanerDelegate {
         // Also check if the HFile is in the HBASE_TEMP_DIRECTORY; this is where the referenced
         // file gets created when cloning a snapshot.
         hfilePath = HFileLink.getHFileFromBackReference(
-            new Path(FSUtils.getRootDir(getConf()), HConstants.HBASE_TEMP_DIRECTORY), filePath);
+          new Path(CommonFSUtils.getRootDir(getConf()), HConstants.HBASE_TEMP_DIRECTORY), filePath);
         if (fs.exists(hfilePath)) {
           return false;
         }
@@ -71,12 +70,13 @@ public class HFileLinkCleaner extends BaseHFileCleanerDelegate {
         if (fs.exists(hfilePath)) {
           return false;
         }
-        hfilePath = HFileLink.getHFileFromBackReference(FSUtils.getRootDir(getConf()), filePath);
+        hfilePath =
+          HFileLink.getHFileFromBackReference(CommonFSUtils.getRootDir(getConf()), filePath);
         return !fs.exists(hfilePath);
       } catch (IOException e) {
         if (LOG.isDebugEnabled()) {
-          LOG.debug("Couldn't verify if the referenced file still exists, keep it just in case: "
-              + hfilePath);
+          LOG.debug("Couldn't verify if the referenced file still exists, keep it just in case: " +
+            hfilePath);
         }
         return false;
       }
@@ -86,7 +86,7 @@ public class HFileLinkCleaner extends BaseHFileCleanerDelegate {
     Path backRefDir = null;
     try {
       backRefDir = HFileLink.getBackReferencesDir(parentDir, filePath.getName());
-      return FSUtils.listStatus(fs, backRefDir) == null;
+      return CommonFSUtils.listStatus(fs, backRefDir) == null;
     } catch (IOException e) {
       if (LOG.isDebugEnabled()) {
         LOG.debug("Couldn't get the references, not deleting file, just in case. filePath="
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineNamespaceProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineNamespaceProcedure.java
index 341d116..e847a2c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineNamespaceProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineNamespaceProcedure.java
@@ -19,8 +19,8 @@
 package org.apache.hadoop.hbase.master.procedure;
 
 import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Base class for all the Namespace procedures that want to use a StateMachineProcedure.
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineTableProcedure.java
index 431b988..3b907f5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineTableProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineTableProcedure.java
@@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.master.TableStateManager;
 import org.apache.hadoop.hbase.master.assignment.RegionStateNode;
 import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
 import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
@@ -131,7 +131,7 @@ public abstract class AbstractStateMachineTableProcedure<TState>
 
   protected final Path getWALRegionDir(MasterProcedureEnv env, RegionInfo region)
       throws IOException {
-    return FSUtils.getWALRegionDir(env.getMasterConfiguration(),
+    return CommonFSUtils.getWALRegionDir(env.getMasterConfiguration(),
         region.getTable(), region.getEncodedName());
   }
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java
index c569de2..99bb26e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java
@@ -24,7 +24,6 @@ import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -51,13 +50,15 @@ import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
 import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper;
 import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
 import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
+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.util.Pair;
 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.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
@@ -456,10 +457,10 @@ public class CloneSnapshotProcedure
 
     // 1. Create Table Descriptor
     // using a copy of descriptor, table will be created enabling first
-    final Path tempTableDir = FSUtils.getTableDir(tempdir, tableDescriptor.getTableName());
-    ((FSTableDescriptors)(env.getMasterServices().getTableDescriptors()))
+    final Path tempTableDir = CommonFSUtils.getTableDir(tempdir, tableDescriptor.getTableName());
+    ((FSTableDescriptors) (env.getMasterServices().getTableDescriptors()))
       .createTableDescriptorForTableDirectory(tempTableDir,
-              TableDescriptorBuilder.newBuilder(tableDescriptor).build(), false);
+        TableDescriptorBuilder.newBuilder(tableDescriptor).build(), false);
 
     // 2. Create Regions
     newRegions = hdfsRegionHandler.createHdfsRegions(
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java
index 2f56e83..0bcb561 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java
@@ -19,19 +19,19 @@
 package org.apache.hadoop.hbase.master.procedure;
 
 import java.io.IOException;
-
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.NamespaceExistException;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hbase.master.MasterFileSystem;
 import org.apache.hadoop.hbase.master.TableNamespaceManager;
 import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+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;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateNamespaceState;
-import org.apache.hadoop.hbase.util.FSUtils;
 
 /**
  * The procedure to create a new namespace.
@@ -225,7 +225,7 @@ public class CreateNamespaceProcedure
       final NamespaceDescriptor nsDescriptor) throws IOException {
     MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
     mfs.getFileSystem().mkdirs(
-      FSUtils.getNamespaceDir(mfs.getRootDir(), nsDescriptor.getName()));
+      CommonFSUtils.getNamespaceDir(mfs.getRootDir(), nsDescriptor.getName()));
   }
 
   /**
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java
index cc7a195..14446df 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.master.procedure;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
-
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
@@ -35,15 +34,17 @@ import org.apache.hadoop.hbase.client.TableState;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.MasterFileSystem;
 import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
+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.util.ModifyRegionUtils;
 import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
 import org.apache.yetus.audience.InterfaceAudience;
 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;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
@@ -308,7 +309,7 @@ public class CreateTableProcedure
 
     // 1. Create Table Descriptor
     // using a copy of descriptor, table will be created enabling first
-    final Path tempTableDir = FSUtils.getTableDir(tempdir, tableDescriptor.getTableName());
+    final Path tempTableDir = CommonFSUtils.getTableDir(tempdir, tableDescriptor.getTableName());
     ((FSTableDescriptors)(env.getMasterServices().getTableDescriptors()))
         .createTableDescriptorForTableDirectory(tempTableDir, tableDescriptor, false);
 
@@ -327,7 +328,8 @@ public class CreateTableProcedure
     final TableDescriptor tableDescriptor,
     final Path tempTableDir) throws IOException {
     final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
-    final Path tableDir = FSUtils.getTableDir(mfs.getRootDir(), tableDescriptor.getTableName());
+    final Path tableDir =
+      CommonFSUtils.getTableDir(mfs.getRootDir(), tableDescriptor.getTableName());
     FileSystem fs = mfs.getFileSystem();
     if (!fs.delete(tableDir, true) && fs.exists(tableDir)) {
       throw new IOException("Couldn't delete " + tableDir);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteNamespaceProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteNamespaceProcedure.java
index 8369a19..7d4ba7f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteNamespaceProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteNamespaceProcedure.java
@@ -20,24 +20,24 @@ package org.apache.hadoop.hbase.master.procedure;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
-
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.NamespaceNotFoundException;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hbase.constraint.ConstraintException;
 import org.apache.hadoop.hbase.master.MasterFileSystem;
 import org.apache.hadoop.hbase.master.TableNamespaceManager;
 import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+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;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteNamespaceState;
-import org.apache.hadoop.hbase.util.FSUtils;
 
 /**
  * The procedure to remove a namespace.
@@ -292,7 +292,7 @@ public class DeleteNamespaceProcedure
       final String namespaceName) throws IOException {
     MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
     FileSystem fs = mfs.getFileSystem();
-    Path p = FSUtils.getNamespaceDir(mfs.getRootDir(), namespaceName);
+    Path p = CommonFSUtils.getNamespaceDir(mfs.getRootDir(), namespaceName);
 
     try {
       for(FileStatus status : fs.listStatus(p)) {
@@ -300,7 +300,7 @@ public class DeleteNamespaceProcedure
           throw new IOException("Namespace directory contains table dir: " + status.getPath());
         }
       }
-      if (!fs.delete(FSUtils.getNamespaceDir(mfs.getRootDir(), namespaceName), true)) {
+      if (!fs.delete(CommonFSUtils.getNamespaceDir(mfs.getRootDir(), namespaceName), true)) {
         throw new IOException("Failed to remove namespace: " + namespaceName);
       }
     } catch (FileNotFoundException e) {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java
index 6c862af..5b118a4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java
@@ -45,10 +45,12 @@ import org.apache.hadoop.hbase.master.MasterFileSystem;
 import org.apache.hadoop.hbase.mob.MobConstants;
 import org.apache.hadoop.hbase.mob.MobUtils;
 import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
@@ -275,8 +277,8 @@ public class DeleteTableProcedure
     final FileSystem fs = mfs.getFileSystem();
     final Path tempdir = mfs.getTempDir();
 
-    final Path tableDir = FSUtils.getTableDir(mfs.getRootDir(), tableName);
-    final Path tempTableDir = FSUtils.getTableDir(tempdir, tableName);
+    final Path tableDir = CommonFSUtils.getTableDir(mfs.getRootDir(), tableName);
+    final Path tempTableDir = CommonFSUtils.getTableDir(tempdir, tableName);
 
     if (fs.exists(tableDir)) {
       // Ensure temp exists
@@ -324,8 +326,8 @@ public class DeleteTableProcedure
     }
 
     // Archive mob data
-    Path mobTableDir = FSUtils.getTableDir(new Path(mfs.getRootDir(), MobConstants.MOB_DIR_NAME),
-            tableName);
+    Path mobTableDir =
+      CommonFSUtils.getTableDir(new Path(mfs.getRootDir(), MobConstants.MOB_DIR_NAME), tableName);
     Path regionDir =
             new Path(mobTableDir, MobUtils.getMobRegionInfo(tableName).getEncodedName());
     if (fs.exists(regionDir)) {
@@ -346,7 +348,7 @@ public class DeleteTableProcedure
 
     // Delete the directory on wal filesystem
     FileSystem walFs = mfs.getWALFileSystem();
-    Path tableWALDir = FSUtils.getWALTableDir(env.getMasterConfiguration(), tableName);
+    Path tableWALDir = CommonFSUtils.getWALTableDir(env.getMasterConfiguration(), tableName);
     if (walFs.exists(tableWALDir) && !walFs.delete(tableWALDir, true)) {
       throw new IOException("Couldn't delete table dir on wal filesystem" + tableWALDir);
     }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java
index a3772fd..292d76e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java
@@ -57,8 +57,7 @@ public class MasterProcedureEnv implements ConfigurationObserver {
     @Override
     public void recoverFileLease(final FileSystem fs, final Path path) throws IOException {
       final Configuration conf = master.getConfiguration();
-      final FSUtils fsUtils = FSUtils.getInstance(fs, conf);
-      fsUtils.recoverFileLease(fs, path, conf, new CancelableProgressable() {
+      FSUtils.recoverFileLease(fs, path, conf, new CancelableProgressable() {
         @Override
         public boolean progress() {
           LOG.debug("Recover Procedure Store log lease: " + path);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/DisabledTableSnapshotHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/DisabledTableSnapshotHandler.java
index 39b8425..c670c51 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/DisabledTableSnapshotHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/DisabledTableSnapshotHandler.java
@@ -22,7 +22,6 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 import java.util.concurrent.ThreadPoolExecutor;
-
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
@@ -31,7 +30,7 @@ import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.mob.MobUtils;
 import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
 import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.ModifyRegionUtils;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -39,6 +38,7 @@ import org.apache.yetus.audience.InterfaceStability;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
 
 /**
@@ -104,7 +104,8 @@ public class DisabledTableSnapshotHandler extends TakeSnapshotHandler {
         ModifyRegionUtils.editRegions(exec, regions, new ModifyRegionUtils.RegionEditTask() {
           @Override
           public void editRegion(final RegionInfo regionInfo) throws IOException {
-            snapshotManifest.addRegion(FSUtils.getTableDir(rootDir, snapshotTable), regionInfo);
+            snapshotManifest.addRegion(CommonFSUtils.getTableDir(rootDir, snapshotTable),
+              regionInfo);
           }
         });
       } finally {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/MasterSnapshotVerifier.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/MasterSnapshotVerifier.java
index c9dc0c2..23d0263 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/MasterSnapshotVerifier.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/MasterSnapshotVerifier.java
@@ -35,7 +35,7 @@ import org.apache.hadoop.hbase.snapshot.CorruptedSnapshotException;
 import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
 import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
 import org.apache.hadoop.hbase.snapshot.SnapshotReferenceUtil;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.yetus.audience.InterfaceStability;
@@ -209,7 +209,7 @@ public final class MasterSnapshotVerifier {
     // Verify Snapshot HFiles
     // Requires the root directory file system as HFiles are stored in the root directory
     SnapshotReferenceUtil.verifySnapshot(services.getConfiguration(),
-        FSUtils.getRootDirFileSystem(services.getConfiguration()), manifest);
+      CommonFSUtils.getRootDirFileSystem(services.getConfiguration()), manifest);
   }
 
   /**
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotFileCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotFileCache.java
index e6c280a..a24c7d4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotFileCache.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotFileCache.java
@@ -34,7 +34,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Stoppable;
 import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.yetus.audience.InterfaceStability;
 import org.slf4j.Logger;
@@ -109,9 +109,9 @@ public class SnapshotFileCache implements Stoppable {
    * @throws IOException if the {@link FileSystem} or root directory cannot be loaded
    */
   public SnapshotFileCache(Configuration conf, long cacheRefreshPeriod, String refreshThreadName,
-      SnapshotFileInspector inspectSnapshotFiles) throws IOException {
-    this(FSUtils.getCurrentFileSystem(conf), FSUtils.getRootDir(conf), 0, cacheRefreshPeriod,
-      refreshThreadName, inspectSnapshotFiles);
+    SnapshotFileInspector inspectSnapshotFiles) throws IOException {
+    this(CommonFSUtils.getCurrentFileSystem(conf), CommonFSUtils.getRootDir(conf), 0,
+      cacheRefreshPeriod, refreshThreadName, inspectSnapshotFiles);
   }
 
   /**
@@ -212,7 +212,7 @@ public class SnapshotFileCache implements Stoppable {
     // just list the snapshot directory directly, do not check the modification time for the root
     // snapshot directory, as some file system implementations do not modify the parent directory's
     // modTime when there are new sub items, for example, S3.
-    FileStatus[] snapshotDirs = FSUtils.listStatus(fs, snapshotDir,
+    FileStatus[] snapshotDirs = CommonFSUtils.listStatus(fs, snapshotDir,
       p -> !p.getName().equals(SnapshotDescriptionUtils.SNAPSHOT_TMP_DIR_NAME));
 
     // clear the cache, as in the below code, either we will also clear the snapshots, or we will
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotHFileCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotHFileCleaner.java
index 559863e..af24f43 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotHFileCleaner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotHFileCleaner.java
@@ -21,11 +21,6 @@ import java.io.IOException;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Map;
-
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.yetus.audience.InterfaceStability;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -36,7 +31,11 @@ import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.master.cleaner.BaseHFileCleanerDelegate;
 import org.apache.hadoop.hbase.snapshot.CorruptedSnapshotException;
 import org.apache.hadoop.hbase.snapshot.SnapshotReferenceUtil;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Implementation of a file cleaner that checks if a hfile is still used by snapshots of HBase
@@ -92,8 +91,8 @@ public class SnapshotHFileCleaner extends BaseHFileCleanerDelegate {
     try {
       long cacheRefreshPeriod = conf.getLong(HFILE_CACHE_REFRESH_PERIOD_CONF_KEY,
         DEFAULT_HFILE_CACHE_REFRESH_PERIOD);
-      final FileSystem fs = FSUtils.getCurrentFileSystem(conf);
-      Path rootDir = FSUtils.getRootDir(conf);
+      final FileSystem fs = CommonFSUtils.getCurrentFileSystem(conf);
+      Path rootDir = CommonFSUtils.getRootDir(conf);
       cache = new SnapshotFileCache(fs, rootDir, cacheRefreshPeriod, cacheRefreshPeriod,
           "snapshot-hfile-cleaner-cache-refresher", new SnapshotFileCache.SnapshotFileInspector() {
             @Override
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java
index e00c749..1e77182 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java
@@ -35,7 +35,6 @@ import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileStatus;
@@ -84,8 +83,8 @@ import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
 import org.apache.hadoop.hbase.snapshot.SnapshotReferenceUtil;
 import org.apache.hadoop.hbase.snapshot.TablePartiallyOpenException;
 import org.apache.hadoop.hbase.snapshot.UnknownSnapshotException;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.NonceKey;
 import org.apache.hadoop.hbase.util.TableDescriptorChecker;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -93,13 +92,15 @@ import org.apache.yetus.audience.InterfaceStability;
 import org.apache.zookeeper.KeeperException;
 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.util.concurrent.ThreadFactoryBuilder;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription.Type;
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
 
 /**
  * This class manages the procedure of taking and restoring snapshots. There is only one
@@ -1158,7 +1159,7 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
       LOG.info("Snapshot feature is not enabled, missing log and hfile cleaners.");
       Path snapshotDir = SnapshotDescriptionUtils.getSnapshotsDir(mfs.getRootDir());
       if (fs.exists(snapshotDir)) {
-        FileStatus[] snapshots = FSUtils.listStatus(fs, snapshotDir,
+        FileStatus[] snapshots = CommonFSUtils.listStatus(fs, snapshotDir,
           new SnapshotDescriptionUtils.CompletedSnaphotDirectoriesFilter(fs));
         if (snapshots != null) {
           LOG.error("Snapshots are present, but cleaners are not enabled.");
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 a5f091b..a77ec4c 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
@@ -50,7 +50,7 @@ import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
 import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
 import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
 import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -59,6 +59,7 @@ 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.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
 
@@ -314,7 +315,7 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh
    */
   protected void snapshotDisabledRegion(final RegionInfo regionInfo)
       throws IOException {
-    snapshotManifest.addRegion(FSUtils.getTableDir(rootDir, snapshotTable), regionInfo);
+    snapshotManifest.addRegion(CommonFSUtils.getTableDir(rootDir, snapshotTable), regionInfo);
     monitor.rethrowException();
     status.setStatus("Completed referencing HFiles for offline region " + regionInfo.toString() +
         " of table: " + snapshotTable);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/ManualMobMaintHFileCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/ManualMobMaintHFileCleaner.java
index 408030d..0e53ce9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/ManualMobMaintHFileCleaner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/ManualMobMaintHFileCleaner.java
@@ -19,13 +19,12 @@ package org.apache.hadoop.hbase.mob;
 
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
-
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.master.cleaner.BaseHFileCleanerDelegate;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -62,7 +61,7 @@ public class ManualMobMaintHFileCleaner extends BaseHFileCleanerDelegate {
       Path region = family.getParent();
       Path table = region.getParent();
 
-      TableName tableName = FSUtils.getTableName(table);
+      TableName tableName = CommonFSUtils.getTableName(table);
 
       String mobRegion = MOB_REGIONS.get(tableName);
       if (mobRegion == null) {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java
index 1c00e25..2a6e637 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java
@@ -70,8 +70,8 @@ import org.apache.hadoop.hbase.regionserver.HStoreFile;
 import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ChecksumType;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.ReflectionUtils;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -306,7 +306,7 @@ public final class MobUtils {
     LOG.info("MOB HFiles older than " + expireDate.toGMTString() + " will be deleted!");
 
     FileStatus[] stats = null;
-    Path mobTableDir = FSUtils.getTableDir(getMobHome(conf), tableName);
+    Path mobTableDir = CommonFSUtils.getTableDir(getMobHome(conf), tableName);
     Path path = getMobFamilyPath(conf, tableName, columnDescriptor.getNameAsString());
     try {
       stats = fs.listStatus(path);
@@ -396,7 +396,7 @@ public final class MobUtils {
    * @return The table dir of the mob file.
    */
   public static Path getMobTableDir(Path rootDir, TableName tableName) {
-    return FSUtils.getTableDir(getMobHome(rootDir), tableName);
+    return CommonFSUtils.getTableDir(getMobHome(rootDir), tableName);
   }
 
   /**
@@ -418,7 +418,7 @@ public final class MobUtils {
    * @return The region dir of the mob files.
    */
   public static Path getMobRegionPath(Path rootDir, TableName tableName) {
-    Path tablePath = FSUtils.getTableDir(getMobHome(rootDir), tableName);
+    Path tablePath = CommonFSUtils.getTableDir(getMobHome(rootDir), tableName);
     RegionInfo regionInfo = getMobRegionInfo(tableName);
     return new Path(tablePath, regionInfo.getEncodedName());
   }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/MobCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/MobCompactor.java
index c5d93ea..8b5fa74 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/MobCompactor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/MobCompactor.java
@@ -22,16 +22,15 @@ import java.io.IOException;
 import java.util.Arrays;
 import java.util.List;
 import java.util.concurrent.ExecutorService;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.mob.MobUtils;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * A mob compactor to directly compact the mob files.
@@ -55,7 +54,7 @@ public abstract class MobCompactor {
     this.tableName = tableName;
     this.column = column;
     this.pool = pool;
-    mobTableDir = FSUtils.getTableDir(MobUtils.getMobHome(conf), tableName);
+    mobTableDir = CommonFSUtils.getTableDir(MobUtils.getMobHome(conf), tableName);
     mobFamilyDir = MobUtils.getMobFamilyPath(conf, tableName, column.getNameAsString());
   }
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/FileArchiverNotifierImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/FileArchiverNotifierImpl.java
index aa91696..d81c1f3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/FileArchiverNotifierImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/FileArchiverNotifierImpl.java
@@ -33,7 +33,6 @@ import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
 import java.util.function.Predicate;
 import java.util.stream.Collectors;
-
 import org.apache.commons.lang3.builder.HashCodeBuilder;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
@@ -47,11 +46,11 @@ import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
 import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.HFileArchiveUtil;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.yetus.audience.InterfaceAudience;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -201,7 +200,7 @@ public class FileArchiverNotifierImpl implements FileArchiverNotifier {
     }
 
     Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(
-        snapshotName, FSUtils.getRootDir(conf));
+        snapshotName, CommonFSUtils.getRootDir(conf));
     SnapshotDescription sd = SnapshotDescriptionUtils.readSnapshotInfo(fs, snapshotDir);
     SnapshotManifest manifest = SnapshotManifest.open(conf, fs, snapshotDir, sd);
     // For each region referenced by the snapshot
@@ -380,7 +379,7 @@ public class FileArchiverNotifierImpl implements FileArchiverNotifier {
    */
   List<SnapshotWithSize> computeSnapshotSizes(List<String> snapshots) throws IOException {
     final List<SnapshotWithSize> snapshotSizes = new ArrayList<>(snapshots.size());
-    final Path rootDir = FSUtils.getRootDir(conf);
+    final Path rootDir = CommonFSUtils.getRootDir(conf);
 
     // Get the map of store file names to store file path for this table
     final Set<String> tableReferencedStoreFiles;
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 578caf1..bea3087 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
@@ -164,6 +164,7 @@ import org.apache.hadoop.hbase.trace.TraceUtil;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.CancelableProgressable;
 import org.apache.hadoop.hbase.util.ClassSize;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.HashedBytes;
@@ -1039,8 +1040,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         // This means we have replayed all the recovered edits and also written out the max sequence
         // id file, let's delete the wrong directories introduced in HBASE-20734, see HBASE-22617
         // for more details.
-        Path wrongRegionWALDir = FSUtils.getWrongWALRegionDir(conf, getRegionInfo().getTable(),
-          getRegionInfo().getEncodedName());
+        Path wrongRegionWALDir = CommonFSUtils.getWrongWALRegionDir(conf,
+          getRegionInfo().getTable(), getRegionInfo().getEncodedName());
         FileSystem walFs = getWalFileSystem();
         if (walFs.exists(wrongRegionWALDir)) {
           if (!walFs.delete(wrongRegionWALDir, true)) {
@@ -1237,11 +1238,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
    * @param tableDescriptor TableDescriptor of the table
    * @param regionInfo encoded name of the region
    * @return The HDFS blocks distribution for the given region.
-   * @throws IOException
    */
   public static HDFSBlocksDistribution computeHDFSBlocksDistribution(Configuration conf,
-      TableDescriptor tableDescriptor, RegionInfo regionInfo) throws IOException {
-    Path tablePath = FSUtils.getTableDir(FSUtils.getRootDir(conf), tableDescriptor.getTableName());
+    TableDescriptor tableDescriptor, RegionInfo regionInfo) throws IOException {
+    Path tablePath =
+      CommonFSUtils.getTableDir(CommonFSUtils.getRootDir(conf), tableDescriptor.getTableName());
     return computeHDFSBlocksDistribution(conf, tableDescriptor, regionInfo, tablePath);
   }
 
@@ -1962,13 +1963,13 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   /** @return the WAL {@link HRegionFileSystem} used by this region */
   HRegionWALFileSystem getRegionWALFileSystem() throws IOException {
     return new HRegionWALFileSystem(conf, getWalFileSystem(),
-        FSUtils.getWALTableDir(conf, htableDescriptor.getTableName()), fs.getRegionInfo());
+      CommonFSUtils.getWALTableDir(conf, htableDescriptor.getTableName()), fs.getRegionInfo());
   }
 
   /** @return the WAL {@link FileSystem} being used by this region */
   FileSystem getWalFileSystem() throws IOException {
     if (walFS == null) {
-      walFS = FSUtils.getWALFileSystem(conf);
+      walFS = CommonFSUtils.getWALFileSystem(conf);
     }
     return walFS;
   }
@@ -1980,8 +1981,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   @VisibleForTesting
   public Path getWALRegionDir() throws IOException {
     if (regionDir == null) {
-      regionDir = FSUtils.getWALRegionDir(conf, getRegionInfo().getTable(),
-          getRegionInfo().getEncodedName());
+      regionDir = CommonFSUtils.getWALRegionDir(conf, getRegionInfo().getTable(),
+        getRegionInfo().getEncodedName());
     }
     return regionDir;
   }
@@ -4414,7 +4415,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
    */
   public void addRegionToSnapshot(SnapshotDescription desc,
       ForeignExceptionSnare exnSnare) throws IOException {
-    Path rootDir = FSUtils.getRootDir(conf);
+    Path rootDir = CommonFSUtils.getRootDir(conf);
     Path snapshotDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(desc, rootDir, conf);
 
     SnapshotManifest manifest = SnapshotManifest.create(conf, getFilesystem(),
@@ -4676,10 +4677,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     if (org.apache.commons.lang3.StringUtils.isBlank(specialRecoveredEditsDirStr)) {
       FileSystem walFS = getWalFileSystem();
       FileSystem rootFS = getFilesystem();
-      Path wrongRegionWALDir = FSUtils.getWrongWALRegionDir(conf, getRegionInfo().getTable(),
+      Path wrongRegionWALDir = CommonFSUtils.getWrongWALRegionDir(conf, getRegionInfo().getTable(),
         getRegionInfo().getEncodedName());
       Path regionWALDir = getWALRegionDir();
-      Path regionDir = FSUtils.getRegionDirFromRootDir(FSUtils.getRootDir(conf), getRegionInfo());
+      Path regionDir =
+        FSUtils.getRegionDirFromRootDir(CommonFSUtils.getRootDir(conf), getRegionInfo());
 
       // We made a mistake in HBASE-20734 so we need to do this dirty hack...
       NavigableSet<Path> filesUnderWrongRegionWALDir =
@@ -7222,7 +7224,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       (hTableDescriptor == null ? "null" : hTableDescriptor) + ", regionDir=" + rootDir);
     createRegionDir(conf, info, rootDir);
     FileSystem fs = rootDir.getFileSystem(conf);
-    Path tableDir = FSUtils.getTableDir(rootDir, info.getTable());
+    Path tableDir = CommonFSUtils.getTableDir(rootDir, info.getTable());
     HRegion region = HRegion.newHRegion(tableDir, wal, fs, conf, info, hTableDescriptor, null);
     if (initialize) {
       region.initialize(null);
@@ -7249,7 +7251,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         Path rootDir)
       throws IOException {
     FileSystem fs = rootDir.getFileSystem(configuration);
-    Path tableDir = FSUtils.getTableDir(rootDir, ri.getTable());
+    Path tableDir = CommonFSUtils.getTableDir(rootDir, ri.getTable());
     // If directory already exists, will log warning and keep going. Will try to create
     // .regioninfo. If one exists, will overwrite.
     return HRegionFileSystem.createRegionOnFileSystem(configuration, fs, tableDir, ri);
@@ -7302,7 +7304,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     final RegionServerServices rsServices,
     final CancelableProgressable reporter)
   throws IOException {
-    return openHRegion(FSUtils.getRootDir(conf), info, htd, wal, conf, rsServices, reporter);
+    return openHRegion(CommonFSUtils.getRootDir(conf), info, htd, wal, conf, rsServices, reporter);
   }
 
   /**
@@ -7392,7 +7394,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     final Path rootDir, final RegionInfo info, final TableDescriptor htd, final WAL wal,
     final RegionServerServices rsServices, final CancelableProgressable reporter)
     throws IOException {
-    Path tableDir = FSUtils.getTableDir(rootDir, info.getTable());
+    Path tableDir = CommonFSUtils.getTableDir(rootDir, info.getTable());
     return openHRegionFromTableDir(conf, fs, tableDir, info, htd, wal, rsServices, reporter);
   }
 
@@ -7514,8 +7516,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       LOG.debug("HRegion.Warming up region: " + info);
     }
 
-    Path rootDir = FSUtils.getRootDir(conf);
-    Path tableDir = FSUtils.getTableDir(rootDir, info.getTable());
+    Path rootDir = CommonFSUtils.getRootDir(conf);
+    Path tableDir = CommonFSUtils.getTableDir(rootDir, info.getTable());
 
     FileSystem fs = null;
     if (rsServices != null) {
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 8402a64..d5ef30e 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
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
+import edu.umd.cs.findbugs.annotations.Nullable;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InterruptedIOException;
@@ -26,7 +27,6 @@ import java.util.List;
 import java.util.Objects;
 import java.util.Optional;
 import java.util.UUID;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -46,17 +46,16 @@ import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.fs.HFileSystem;
 import org.apache.hadoop.hbase.io.Reference;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSHDFSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
-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.collect.Lists;
 
-import edu.umd.cs.findbugs.annotations.Nullable;
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 
 /**
  * View to an on-disk Region.
@@ -192,7 +191,7 @@ public class HRegionFileSystem {
    * 'COLD', 'WARM', 'HOT', 'ONE_SSD', 'ALL_SSD', 'LAZY_PERSIST'.
    */
   public void setStoragePolicy(String familyName, String policyName) {
-    FSUtils.setStoragePolicy(this.fs, getStoreDir(familyName), policyName);
+    CommonFSUtils.setStoragePolicy(this.fs, getStoreDir(familyName), policyName);
   }
 
   /**
@@ -234,7 +233,7 @@ public class HRegionFileSystem {
   public Collection<StoreFileInfo> getStoreFiles(final String familyName, final boolean validate)
       throws IOException {
     Path familyDir = getStoreDir(familyName);
-    FileStatus[] files = FSUtils.listStatus(this.fs, familyDir);
+    FileStatus[] files = CommonFSUtils.listStatus(this.fs, familyDir);
     if (files == null) {
       if (LOG.isTraceEnabled()) {
         LOG.trace("No StoreFiles for: " + familyDir);
@@ -270,7 +269,7 @@ public class HRegionFileSystem {
       final HRegionFileSystem regionfs, final String familyName,
       final boolean validate) throws IOException {
     Path familyDir = regionfs.getStoreDir(familyName);
-    List<LocatedFileStatus> locatedFileStatuses = FSUtils.listLocatedStatus(
+    List<LocatedFileStatus> locatedFileStatuses = CommonFSUtils.listLocatedStatus(
         regionfs.getFileSystem(), familyDir);
     if (locatedFileStatuses == null) {
       if (LOG.isTraceEnabled()) {
@@ -328,7 +327,7 @@ public class HRegionFileSystem {
    */
   public boolean hasReferences(final String familyName) throws IOException {
     Path storeDir = getStoreDir(familyName);
-    FileStatus[] files = FSUtils.listStatus(fs, storeDir);
+    FileStatus[] files = CommonFSUtils.listStatus(fs, storeDir);
     if (files != null) {
       for(FileStatus stat: files) {
         if(stat.isDirectory()) {
@@ -363,11 +362,12 @@ public class HRegionFileSystem {
    * @throws IOException
    */
   public Collection<String> getFamilies() throws IOException {
-    FileStatus[] fds = FSUtils.listStatus(fs, getRegionDir(), new FSUtils.FamilyDirFilter(fs));
+    FileStatus[] fds =
+      CommonFSUtils.listStatus(fs, getRegionDir(), new FSUtils.FamilyDirFilter(fs));
     if (fds == null) return null;
 
     ArrayList<String> families = new ArrayList<>(fds.length);
-    for (FileStatus status: fds) {
+    for (FileStatus status : fds) {
       families.add(status.getPath().getName());
     }
 
@@ -536,7 +536,7 @@ public class HRegionFileSystem {
     // We can't compare FileSystem instances as equals() includes UGI instance
     // as part of the comparison and won't work when doing SecureBulkLoad
     // TODO deal with viewFS
-    if (!FSHDFSUtils.isSameHdfs(conf, realSrcFs, desFs)) {
+    if (!FSUtils.isSameHdfs(conf, realSrcFs, desFs)) {
       LOG.info("Bulk-load file " + srcPath + " is on different filesystem than " +
           "the destination store. Copying file over to destination filesystem.");
       Path tmpPath = createTempName();
@@ -583,7 +583,7 @@ public class HRegionFileSystem {
     // where we successfully created daughter a but regionserver crashed during
     // the creation of region b.  In this case, there'll be an orphan daughter
     // dir in the filesystem.  TOOD: Fix.
-    FileStatus[] daughters = FSUtils.listStatus(fs, splitdir, new FSUtils.DirFilter(fs));
+    FileStatus[] daughters = CommonFSUtils.listStatus(fs, splitdir, new FSUtils.DirFilter(fs));
     if (daughters != null) {
       for (FileStatus daughter: daughters) {
         Path daughterDir = new Path(getTableDir(), daughter.getPath().getName());
@@ -758,10 +758,10 @@ public class HRegionFileSystem {
 
   static boolean mkdirs(FileSystem fs, Configuration conf, Path dir) throws IOException {
     if (FSUtils.isDistributedFileSystem(fs) ||
-        !conf.getBoolean(HConstants.ENABLE_DATA_FILE_UMASK, false)) {
+      !conf.getBoolean(HConstants.ENABLE_DATA_FILE_UMASK, false)) {
       return fs.mkdirs(dir);
     }
-    FsPermission perms = FSUtils.getFilePermissions(fs, conf, HConstants.DATA_FILE_UMASK_KEY);
+    FsPermission perms = CommonFSUtils.getFilePermissions(fs, conf, HConstants.DATA_FILE_UMASK_KEY);
     return fs.mkdirs(dir, perms);
   }
 
@@ -845,7 +845,7 @@ public class HRegionFileSystem {
    * @throws IOException if an unexpected exception occurs
    */
   void logFileSystemState(final Logger LOG) throws IOException {
-    FSUtils.logFileSystemState(fs, this.getRegionDir(), LOG);
+    CommonFSUtils.logFileSystemState(fs, this.getRegionDir(), LOG);
   }
 
   /**
@@ -876,18 +876,16 @@ public class HRegionFileSystem {
 
   /**
    * Write the .regioninfo file on-disk.
+   * <p/>
    * Overwrites if exists already.
    */
   private static void writeRegionInfoFileContent(final Configuration conf, final FileSystem fs,
-      final Path regionInfoFile, final byte[] content) throws IOException {
+    final Path regionInfoFile, final byte[] content) throws IOException {
     // First check to get the permissions
-    FsPermission perms = FSUtils.getFilePermissions(fs, conf, HConstants.DATA_FILE_UMASK_KEY);
+    FsPermission perms = CommonFSUtils.getFilePermissions(fs, conf, HConstants.DATA_FILE_UMASK_KEY);
     // Write the RegionInfo file content
-    FSDataOutputStream out = FSUtils.create(conf, fs, regionInfoFile, perms, null);
-    try {
+    try (FSDataOutputStream out = FSUtils.create(conf, fs, regionInfoFile, perms, null)) {
       out.write(content);
-    } finally {
-      out.close();
     }
   }
 
@@ -965,8 +963,8 @@ public class HRegionFileSystem {
       // close the created regioninfo file in the .tmp directory then on next
       // creation we will be getting AlreadyCreatedException.
       // Hence delete and create the file if exists.
-      if (FSUtils.isExists(fs, tmpPath)) {
-        FSUtils.delete(fs, tmpPath, true);
+      if (CommonFSUtils.isExists(fs, tmpPath)) {
+        CommonFSUtils.delete(fs, tmpPath, true);
       }
 
       // Write HRI to a file in case we need to recover hbase:meta
@@ -1079,7 +1077,7 @@ public class HRegionFileSystem {
     }
 
     // Archive region
-    Path rootDir = FSUtils.getRootDir(conf);
+    Path rootDir = CommonFSUtils.getRootDir(conf);
     HFileArchiver.archiveRegion(fs, rootDir, tableDir, regionDir);
 
     // Delete empty region dir
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 904447c..87fbec2 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
@@ -153,6 +153,7 @@ import org.apache.hadoop.hbase.trace.SpanReceiverHost;
 import org.apache.hadoop.hbase.trace.TraceUtil;
 import org.apache.hadoop.hbase.util.Addressing;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.CompressionTest;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSTableDescriptors;
@@ -717,15 +718,15 @@ public class HRegionServer extends HasThread implements
     // Get fs instance used by this RS.  Do we use checksum verification in the hbase? If hbase
     // checksum verification enabled, then automatically switch off hdfs checksum verification.
     boolean useHBaseChecksum = conf.getBoolean(HConstants.HBASE_CHECKSUM_VERIFICATION, true);
-    FSUtils.setFsDefault(this.conf, FSUtils.getWALRootDir(this.conf));
+    CommonFSUtils.setFsDefault(this.conf, CommonFSUtils.getWALRootDir(this.conf));
     this.walFs = new HFileSystem(this.conf, useHBaseChecksum);
-    this.walRootDir = FSUtils.getWALRootDir(this.conf);
+    this.walRootDir = CommonFSUtils.getWALRootDir(this.conf);
     // Set 'fs.defaultFS' to match the filesystem on hbase.rootdir else
     // underlying hadoop hdfs accessors will be going against wrong filesystem
     // (unless all is set to defaults).
-    FSUtils.setFsDefault(this.conf, FSUtils.getRootDir(this.conf));
+    CommonFSUtils.setFsDefault(this.conf, CommonFSUtils.getRootDir(this.conf));
     this.dataFs = new HFileSystem(this.conf, useHBaseChecksum);
-    this.dataRootDir = FSUtils.getRootDir(this.conf);
+    this.dataRootDir = CommonFSUtils.getRootDir(this.conf);
     this.tableDescriptors =
         new FSTableDescriptors(this.dataFs, this.dataRootDir, !canUpdateTableDescriptor(), false);
     if (this instanceof HMaster) {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadManager.java
index ff9b229..40ba388 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SecureBulkLoadManager.java
@@ -47,7 +47,7 @@ import org.apache.hadoop.hbase.security.token.AuthenticationTokenIdentifier;
 import org.apache.hadoop.hbase.security.token.ClientTokenUtil;
 import org.apache.hadoop.hbase.security.token.FsDelegationToken;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSHDFSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.Methods;
 import org.apache.hadoop.hbase.util.Pair;
@@ -124,7 +124,7 @@ public class SecureBulkLoadManager {
     userProvider = UserProvider.instantiate(conf);
     ugiReferenceCounter = new ConcurrentHashMap<>();
     fs = FileSystem.get(conf);
-    baseStagingDir = new Path(FSUtils.getRootDir(conf), HConstants.BULKLOAD_STAGING_DIR_NAME);
+    baseStagingDir = new Path(CommonFSUtils.getRootDir(conf), HConstants.BULKLOAD_STAGING_DIR_NAME);
 
     if (conf.get("hbase.bulkload.staging.dir") != null) {
       LOG.warn("hbase.bulkload.staging.dir " + " is deprecated. Bulkload staging directory is "
@@ -382,7 +382,7 @@ public class SecureBulkLoadManager {
       }
 
       // Check to see if the source and target filesystems are the same
-      if (!FSHDFSUtils.isSameHdfs(conf, srcFs, fs)) {
+      if (!FSUtils.isSameHdfs(conf, srcFs, fs)) {
         LOG.debug("Bulk-load file " + srcPath + " is on different filesystem than " +
             "the destination filesystem. Copying file over to destination staging dir.");
         FileUtil.copy(srcFs, p, fs, stageP, false, conf);
@@ -421,7 +421,7 @@ public class SecureBulkLoadManager {
         if (srcFs == null) {
           srcFs = FileSystem.newInstance(p.toUri(), conf);
         }
-        if (!FSHDFSUtils.isSameHdfs(conf, srcFs, fs)) {
+        if (!FSUtils.isSameHdfs(conf, srcFs, fs)) {
           // files are copied so no need to move them back
           return;
         }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitLogWorker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitLogWorker.java
index 1cb0f35..e842826 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitLogWorker.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitLogWorker.java
@@ -23,10 +23,6 @@ import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.net.ConnectException;
 import java.net.SocketTimeoutException;
-
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -35,11 +31,14 @@ import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.client.RetriesExhaustedException;
 import org.apache.hadoop.hbase.coordination.SplitLogWorkerCoordination;
 import org.apache.hadoop.hbase.regionserver.SplitLogWorker.TaskExecutor.Status;
-import org.apache.hadoop.hbase.wal.WALFactory;
-import org.apache.hadoop.hbase.wal.WALSplitter;
 import org.apache.hadoop.hbase.util.CancelableProgressable;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.ExceptionUtil;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.wal.WALFactory;
+import org.apache.hadoop.hbase.wal.WALSplitter;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 
@@ -89,7 +88,7 @@ public class SplitLogWorker implements Runnable {
     Path walDir;
     FileSystem fs;
     try {
-      walDir = FSUtils.getWALRootDir(conf);
+      walDir = CommonFSUtils.getWALRootDir(conf);
       fs = walDir.getFileSystem(conf);
     } catch (IOException e) {
       LOG.warn("could not find root dir or fs", e);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java
index 02665a3..01fdeba 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java
@@ -26,6 +26,7 @@ import static org.apache.hadoop.hbase.regionserver.HStoreFile.MAJOR_COMPACTION_K
 import static org.apache.hadoop.hbase.regionserver.HStoreFile.MAX_SEQ_ID_KEY;
 import static org.apache.hadoop.hbase.regionserver.HStoreFile.MOB_CELLS_COUNT;
 import static org.apache.hadoop.hbase.regionserver.HStoreFile.TIMERANGE_KEY;
+
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.util.Collection;
@@ -52,14 +53,16 @@ import org.apache.hadoop.hbase.util.BloomFilterFactory;
 import org.apache.hadoop.hbase.util.BloomFilterUtil;
 import org.apache.hadoop.hbase.util.BloomFilterWriter;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.RowBloomContext;
 import org.apache.hadoop.hbase.util.RowColBloomContext;
 import org.apache.hadoop.hbase.util.RowPrefixFixedLengthBloomContext;
 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.ProtobufUtil;
 
 /**
@@ -524,7 +527,7 @@ public class StoreFileWriter implements CellSink, ShipperListener {
       if (null == policyName) {
         policyName = this.conf.get(HStore.BLOCK_STORAGE_POLICY_KEY);
       }
-      FSUtils.setStoragePolicy(this.fs, dir, policyName);
+      CommonFSUtils.setStoragePolicy(this.fs, dir, policyName);
 
       if (filePath == null) {
         filePath = getUniqueFile(fs, dir);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
index 24a529a..d0d2dcd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
@@ -68,7 +68,6 @@ import org.apache.hadoop.hbase.trace.TraceUtil;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
 import org.apache.hadoop.hbase.wal.WAL;
@@ -1200,7 +1199,7 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
   abstract int getLogReplication();
 
   private static void split(final Configuration conf, final Path p) throws IOException {
-    FileSystem fs = FSUtils.getWALFileSystem(conf);
+    FileSystem fs = CommonFSUtils.getWALFileSystem(conf);
     if (!fs.exists(p)) {
       throw new FileNotFoundException(p.toString());
     }
@@ -1208,7 +1207,7 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
       throw new IOException(p + " is not a directory");
     }
 
-    final Path baseDir = FSUtils.getWALRootDir(conf);
+    final Path baseDir = CommonFSUtils.getWALRootDir(conf);
     Path archiveDir = new Path(baseDir, HConstants.HREGION_OLDLOGDIR_NAME);
     if (conf.getBoolean(AbstractFSWALProvider.SEPARATE_OLDLOGDIR,
       AbstractFSWALProvider.DEFAULT_SEPARATE_OLDLOGDIR)) {
@@ -1250,7 +1249,7 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
       for (int i = 1; i < args.length; i++) {
         try {
           Path logPath = new Path(args[i]);
-          FSUtils.setFsDefault(conf, logPath);
+          CommonFSUtils.setFsDefault(conf, logPath);
           split(conf, logPath);
         } catch (IOException t) {
           t.printStackTrace(System.err);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractProtobufLogWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractProtobufLogWriter.java
index ff2864d..a94e5c8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractProtobufLogWriter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractProtobufLogWriter.java
@@ -25,17 +25,12 @@ import java.io.OutputStream;
 import java.security.Key;
 import java.security.SecureRandom;
 import java.util.concurrent.atomic.AtomicLong;
-
 import javax.crypto.spec.SecretKeySpec;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hbase.codec.Codec;
 import org.apache.hadoop.hbase.io.crypto.Cipher;
 import org.apache.hadoop.hbase.io.crypto.Encryption;
@@ -43,12 +38,17 @@ import org.apache.hadoop.hbase.io.crypto.Encryptor;
 import org.apache.hadoop.hbase.io.util.LRUDictionary;
 import org.apache.hadoop.hbase.security.EncryptionUtil;
 import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils.StreamLacksCapabilityException;
+import org.apache.hadoop.hbase.util.EncryptionTest;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
+
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALHeader;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALTrailer;
-import org.apache.hadoop.hbase.util.CommonFSUtils.StreamLacksCapabilityException;
-import org.apache.hadoop.hbase.util.EncryptionTest;
-import org.apache.hadoop.hbase.util.FSUtils;
 
 /**
  * Base class for Protobuf log writer.
@@ -144,8 +144,8 @@ public abstract class AbstractProtobufLogWriter {
     boolean doCompress = conf.getBoolean(HConstants.ENABLE_WAL_COMPRESSION, false);
     if (doCompress) {
       try {
-        this.compressionContext = new CompressionContext(LRUDictionary.class,
-            FSUtils.isRecoveredEdits(path),
+        this.compressionContext =
+          new CompressionContext(LRUDictionary.class, CommonFSUtils.isRecoveredEdits(path),
             conf.getBoolean(CompressionContext.ENABLE_WAL_TAGS_COMPRESSION, true));
       } catch (Exception e) {
         throw new IOException("Failed to initiate CompressionContext", e);
@@ -159,9 +159,9 @@ public abstract class AbstractProtobufLogWriter {
     this.conf = conf;
     boolean doCompress = initializeCompressionContext(conf, path);
     this.trailerWarnSize = conf.getInt(WAL_TRAILER_WARN_SIZE, DEFAULT_WAL_TRAILER_WARN_SIZE);
-    int bufferSize = FSUtils.getDefaultBufferSize(fs);
+    int bufferSize = CommonFSUtils.getDefaultBufferSize(fs);
     short replication = (short) conf.getInt("hbase.regionserver.hlog.replication",
-      FSUtils.getDefaultReplication(fs, path));
+      CommonFSUtils.getDefaultReplication(fs, path));
 
     initOutput(fs, path, overwritable, bufferSize, replication, blocksize);
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
index 1215551..e21e141 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
@@ -47,7 +47,7 @@ import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.trace.TraceUtil;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ClassSize;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.HasThread;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.wal.FSHLogProvider;
@@ -212,7 +212,7 @@ public class FSHLog extends AbstractFSWAL<Writer> {
       final boolean failIfWALExists, final String prefix, final String suffix) throws IOException {
     super(fs, rootDir, logDir, archiveDir, conf, listeners, failIfWALExists, prefix, suffix);
     this.minTolerableReplication = conf.getInt("hbase.regionserver.hlog.tolerable.lowreplication",
-      FSUtils.getDefaultReplication(fs, this.walDir));
+      CommonFSUtils.getDefaultReplication(fs, this.walDir));
     this.lowReplicationRollLimit = conf.getInt("hbase.regionserver.hlog.lowreplication.rolllimit",
       5);
     this.closeErrorsTolerated = conf.getInt("hbase.regionserver.logroll.errors.tolerated", 2);
@@ -422,7 +422,7 @@ public class FSHLog extends AbstractFSWAL<Writer> {
     }
 
     if (LOG.isDebugEnabled()) {
-      LOG.debug("Closing WAL writer in " + FSUtils.getPath(walDir));
+      LOG.debug("Closing WAL writer in " + CommonFSUtils.getPath(walDir));
     }
     if (this.writer != null) {
       this.writer.close();
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ReaderBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ReaderBase.java
index 27d40b2..9b6d69a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ReaderBase.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ReaderBase.java
@@ -20,7 +20,6 @@
 package org.apache.hadoop.hbase.regionserver.wal;
 
 import java.io.IOException;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -28,7 +27,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.io.util.LRUDictionary;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -70,7 +69,7 @@ public abstract class ReaderBase implements AbstractFSWALProvider.Reader {
       try {
         if (compressionContext == null) {
           compressionContext = new CompressionContext(LRUDictionary.class,
-              FSUtils.isRecoveredEdits(path), hasTagCompression());
+            CommonFSUtils.isRecoveredEdits(path), hasTagCompression());
         } else {
           compressionContext.clear();
         }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java
index efd742d..2731de8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java
@@ -38,7 +38,6 @@ import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
-
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
@@ -55,7 +54,7 @@ import org.apache.hadoop.hbase.protobuf.ReplicationProtbufUtil;
 import org.apache.hadoop.hbase.replication.HBaseReplicationEndpoint;
 import org.apache.hadoop.hbase.replication.regionserver.ReplicationSinkManager.SinkPeer;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
 import org.apache.hadoop.ipc.RemoteException;
@@ -67,7 +66,6 @@ import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesti
 import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
 import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
 
-
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface;
 
 /**
@@ -185,7 +183,7 @@ public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoi
       replicationClusterId = this.conf.get(HConstants.REPLICATION_CLUSTER_ID);
     }
     // Construct base namespace directory and hfile archive directory path
-    Path rootDir = FSUtils.getRootDir(conf);
+    Path rootDir = CommonFSUtils.getRootDir(conf);
     Path baseNSDir = new Path(HConstants.BASE_NAMESPACE_DIR);
     baseNamespaceDir = new Path(rootDir, baseNSDir);
     hfileArchiveDir = new Path(rootDir, new Path(HConstants.HFILE_ARCHIVE_DIRECTORY, baseNSDir));
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HFileReplicator.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HFileReplicator.java
index a81c576..7a93d90 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HFileReplicator.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HFileReplicator.java
@@ -45,12 +45,13 @@ import org.apache.hadoop.hbase.security.token.FsDelegationToken;
 import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
 import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles.LoadQueueItem;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
 
 /**
@@ -103,7 +104,8 @@ public class HFileReplicator implements Closeable {
 
     userProvider = UserProvider.instantiate(conf);
     fsDelegationToken = new FsDelegationToken(userProvider, "renewer");
-    this.hbaseStagingDir = new Path(FSUtils.getRootDir(conf), HConstants.BULKLOAD_STAGING_DIR_NAME);
+    this.hbaseStagingDir =
+      new Path(CommonFSUtils.getRootDir(conf), HConstants.BULKLOAD_STAGING_DIR_NAME);
     this.maxCopyThreads =
         this.conf.getInt(REPLICATION_BULKLOAD_COPY_MAXTHREADS_KEY,
           REPLICATION_BULKLOAD_COPY_MAXTHREADS_DEFAULT);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java
index a21ca44..72dfe51 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java
@@ -29,7 +29,7 @@ import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.replication.ReplicationPeer;
 import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -85,7 +85,7 @@ public class RecoveredReplicationSource extends ReplicationSource {
         // to look at)
         List<ServerName> deadRegionServers = this.replicationQueueInfo.getDeadRegionServers();
         LOG.info("NB dead servers : " + deadRegionServers.size());
-        final Path walDir = FSUtils.getWALRootDir(conf);
+        final Path walDir = CommonFSUtils.getWALRootDir(conf);
         for (ServerName curDeadServerName : deadRegionServers) {
           final Path deadRsDirectory =
               new Path(walDir, AbstractFSWALProvider.getWALDirectoryName(curDeadServerName
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
index 7079b90..5e3a09f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
@@ -31,7 +31,7 @@ import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
@@ -75,8 +75,8 @@ public class ReplicationSyncUp extends Configured implements Tool {
     Configuration conf = getConf();
     try (ZKWatcher zkw =
       new ZKWatcher(conf, "syncupReplication" + System.currentTimeMillis(), abortable, true)) {
-      Path walRootDir = FSUtils.getWALRootDir(conf);
-      FileSystem fs = FSUtils.getWALFileSystem(conf);
+      Path walRootDir = CommonFSUtils.getWALRootDir(conf);
+      FileSystem fs = CommonFSUtils.getWALFileSystem(conf);
       Path oldLogDir = new Path(walRootDir, HConstants.HREGION_OLDLOGDIR_NAME);
       Path logDir = new Path(walRootDir, HConstants.HREGION_LOGDIR_NAME);
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntryStream.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntryStream.java
index 6df8f1d..e8ae3e5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntryStream.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntryStream.java
@@ -383,10 +383,8 @@ class WALEntryStream implements Closeable {
   // For HBASE-15019
   private void recoverLease(final Configuration conf, final Path path) {
     try {
-
       final FileSystem dfs = CommonFSUtils.getWALFileSystem(conf);
-      FSUtils fsUtils = FSUtils.getInstance(dfs, conf);
-      fsUtils.recoverFileLease(dfs, path, conf, new CancelableProgressable() {
+      FSUtils.recoverFileLease(dfs, path, conf, new CancelableProgressable() {
         @Override
         public boolean progress() {
           LOG.debug("recover WAL lease: " + path);
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 43d892f..bd5388c 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
@@ -33,7 +33,6 @@ import java.util.Map.Entry;
 import java.util.Set;
 import java.util.TreeMap;
 import java.util.concurrent.ThreadPoolExecutor;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -60,6 +59,7 @@ import org.apache.hadoop.hbase.security.access.Permission;
 import org.apache.hadoop.hbase.security.access.ShadedAccessControlUtil;
 import org.apache.hadoop.hbase.security.access.TablePermission;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.ModifyRegionUtils;
 import org.apache.hadoop.hbase.util.Pair;
@@ -67,7 +67,9 @@ import org.apache.hadoop.io.IOUtils;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.com.google.common.collect.ListMultimap;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
@@ -146,15 +148,10 @@ public class RestoreSnapshotHelper {
     this(conf, fs, manifest, tableDescriptor, rootDir, monitor, status, true);
   }
 
-  public RestoreSnapshotHelper(final Configuration conf,
-      final FileSystem fs,
-      final SnapshotManifest manifest,
-      final TableDescriptor tableDescriptor,
-      final Path rootDir,
-      final ForeignExceptionDispatcher monitor,
-      final MonitoredTask status,
-      final boolean createBackRefs)
-  {
+  public RestoreSnapshotHelper(final Configuration conf, final FileSystem fs,
+    final SnapshotManifest manifest, final TableDescriptor tableDescriptor, final Path rootDir,
+    final ForeignExceptionDispatcher monitor, final MonitoredTask status,
+    final boolean createBackRefs) {
     this.fs = fs;
     this.conf = conf;
     this.snapshotManifest = manifest;
@@ -162,7 +159,7 @@ public class RestoreSnapshotHelper {
     this.snapshotTable = TableName.valueOf(snapshotDesc.getTable());
     this.tableDesc = tableDescriptor;
     this.rootDir = rootDir;
-    this.tableDir = FSUtils.getTableDir(rootDir, tableDesc.getTableName());
+    this.tableDir = CommonFSUtils.getTableDir(rootDir, tableDesc.getTableName());
     this.monitor = monitor;
     this.status = status;
     this.createBackRefs = createBackRefs;
@@ -565,8 +562,10 @@ public class RestoreSnapshotHelper {
    * @return The set of files in the specified family directory.
    */
   private Set<String> getTableRegionFamilyFiles(final Path familyDir) throws IOException {
-    FileStatus[] hfiles = FSUtils.listStatus(fs, familyDir);
-    if (hfiles == null) return Collections.emptySet();
+    FileStatus[] hfiles = CommonFSUtils.listStatus(fs, familyDir);
+    if (hfiles == null) {
+      return Collections.emptySet();
+    }
 
     Set<String> familyFiles = new HashSet<>(hfiles.length);
     for (int i = 0; i < hfiles.length; ++i) {
@@ -807,8 +806,11 @@ public class RestoreSnapshotHelper {
    */
   private List<RegionInfo> getTableRegions() throws IOException {
     LOG.debug("get table regions: " + tableDir);
-    FileStatus[] regionDirs = FSUtils.listStatus(fs, tableDir, new FSUtils.RegionDirFilter(fs));
-    if (regionDirs == null) return null;
+    FileStatus[] regionDirs =
+      CommonFSUtils.listStatus(fs, tableDir, new FSUtils.RegionDirFilter(fs));
+    if (regionDirs == null) {
+      return null;
+    }
 
     List<RegionInfo> regions = new ArrayList<>(regionDirs.length);
     for (int i = 0; i < regionDirs.length; ++i) {
@@ -816,7 +818,7 @@ public class RestoreSnapshotHelper {
       regions.add(hri);
     }
     LOG.debug("found " + regions.size() + " regions for table=" +
-        tableDesc.getTableName().getNameAsString());
+      tableDesc.getTableName().getNameAsString());
     return regions;
   }
 
@@ -857,7 +859,7 @@ public class RestoreSnapshotHelper {
 
     if (LOG.isDebugEnabled()) {
       LOG.debug("Restored table dir:" + restoreDir);
-      FSUtils.logFileSystemState(fs, restoreDir, LOG);
+      CommonFSUtils.logFileSystemState(fs, restoreDir, LOG);
     }
     return metaChanges;
   }
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 bbe2042..01c6d42 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
@@ -21,7 +21,6 @@ import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
 import java.util.Collections;
 import java.util.concurrent.TimeUnit;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -37,6 +36,7 @@ import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.access.PermissionStorage;
 import org.apache.hadoop.hbase.security.access.ShadedAccessControlUtil;
 import org.apache.hadoop.hbase.security.access.UserPermission;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -44,6 +44,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.hbase.thirdparty.com.google.common.collect.ListMultimap;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
 
@@ -278,7 +279,7 @@ public final class SnapshotDescriptionUtils {
    */
   public static boolean isWithinDefaultWorkingDir(final Path workingDir, Configuration conf)
     throws IOException {
-    Path defaultWorkingDir = getDefaultWorkingSnapshotDir(FSUtils.getRootDir(conf));
+    Path defaultWorkingDir = getDefaultWorkingSnapshotDir(CommonFSUtils.getRootDir(conf));
     return workingDir.equals(defaultWorkingDir) || isSubDirectoryOf(workingDir, defaultWorkingDir);
   }
 
@@ -353,11 +354,11 @@ public final class SnapshotDescriptionUtils {
    */
   public static void writeSnapshotInfo(SnapshotDescription snapshot, Path workingDir, FileSystem fs)
       throws IOException {
-    FsPermission perms = FSUtils.getFilePermissions(fs, fs.getConf(),
+    FsPermission perms = CommonFSUtils.getFilePermissions(fs, fs.getConf(),
       HConstants.DATA_FILE_UMASK_KEY);
     Path snapshotInfo = new Path(workingDir, SnapshotDescriptionUtils.SNAPSHOTINFO_FILE);
     try {
-      FSDataOutputStream out = FSUtils.create(fs, snapshotInfo, perms, true);
+      FSDataOutputStream out = CommonFSUtils.create(fs, snapshotInfo, perms, true);
       try {
         snapshot.writeTo(out);
       } finally {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotInfo.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotInfo.java
index 78c7050..c3e0c10 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotInfo.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotInfo.java
@@ -31,7 +31,6 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -42,13 +41,15 @@ import org.apache.hadoop.hbase.client.SnapshotDescription;
 import org.apache.hadoop.hbase.io.HFileLink;
 import org.apache.hadoop.hbase.io.WALLink;
 import org.apache.hadoop.hbase.util.AbstractHBaseTool;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
 import org.apache.hbase.thirdparty.org.apache.commons.cli.Option;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
@@ -272,7 +273,7 @@ public final class SnapshotInfo extends AbstractHBaseTool {
         Path parentDir = filePath.getParent();
         Path backRefDir = HFileLink.getBackReferencesDir(parentDir, filePath.getName());
         try {
-          if (FSUtils.listStatus(fs, backRefDir) == null) {
+          if (CommonFSUtils.listStatus(fs, backRefDir) == null) {
             return false;
           }
         } catch (IOException e) {
@@ -367,8 +368,8 @@ public final class SnapshotInfo extends AbstractHBaseTool {
   public int doWork() throws IOException, InterruptedException {
     if (remoteDir != null) {
       URI defaultFs = remoteDir.getFileSystem(conf).getUri();
-      FSUtils.setFsDefault(conf, new Path(defaultFs));
-      FSUtils.setRootDir(conf, remoteDir);
+      CommonFSUtils.setFsDefault(conf, new Path(defaultFs));
+      CommonFSUtils.setRootDir(conf, remoteDir);
     }
 
     // List Available Snapshots
@@ -384,7 +385,7 @@ public final class SnapshotInfo extends AbstractHBaseTool {
       return 0;
     }
 
-    rootDir = FSUtils.getRootDir(conf);
+    rootDir = CommonFSUtils.getRootDir(conf);
     fs = FileSystem.get(rootDir.toUri(), conf);
     LOG.debug("fs=" + fs.getUri().toString() + " root=" + rootDir);
 
@@ -568,7 +569,7 @@ public final class SnapshotInfo extends AbstractHBaseTool {
   public static SnapshotStats getSnapshotStats(final Configuration conf,
       final SnapshotProtos.SnapshotDescription snapshotDesc,
       final Map<Path, Integer> filesMap) throws IOException {
-    Path rootDir = FSUtils.getRootDir(conf);
+    Path rootDir = CommonFSUtils.getRootDir(conf);
     FileSystem fs = FileSystem.get(rootDir.toUri(), conf);
     Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotDesc, rootDir);
     SnapshotManifest manifest = SnapshotManifest.open(conf, fs, snapshotDir, snapshotDesc);
@@ -592,7 +593,7 @@ public final class SnapshotInfo extends AbstractHBaseTool {
    */
   public static List<SnapshotDescription> getSnapshotList(final Configuration conf)
       throws IOException {
-    Path rootDir = FSUtils.getRootDir(conf);
+    Path rootDir = CommonFSUtils.getRootDir(conf);
     FileSystem fs = FileSystem.get(rootDir.toUri(), conf);
     Path snapshotDir = SnapshotDescriptionUtils.getSnapshotsDir(rootDir);
     FileStatus[] snapshots = fs.listStatus(snapshotDir,
@@ -623,7 +624,7 @@ public final class SnapshotInfo extends AbstractHBaseTool {
       final AtomicLong uniqueHFilesMobSize) throws IOException {
     SnapshotProtos.SnapshotDescription snapshotDesc =
         ProtobufUtil.createHBaseProtosSnapshotDesc(snapshot);
-    Path rootDir = FSUtils.getRootDir(conf);
+    Path rootDir = CommonFSUtils.getRootDir(conf);
     final FileSystem fs = FileSystem.get(rootDir.toUri(), conf);
 
     Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotDesc, rootDir);
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 44430e7..bd0900b 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
@@ -27,7 +27,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -45,15 +44,17 @@ import org.apache.hadoop.hbase.regionserver.HStore;
 import org.apache.hadoop.hbase.regionserver.HStoreFile;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
 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.util.Threads;
 import org.apache.yetus.audience.InterfaceAudience;
 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.protobuf.CodedInputStream;
 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.SnapshotProtos.SnapshotDataManifest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
@@ -287,7 +288,7 @@ public final class SnapshotManifest {
       Path baseDir = tableDir;
       // Open the RegionFS
       if (isMobRegion) {
-        baseDir = FSUtils.getTableDir(MobUtils.getMobHome(conf), regionInfo.getTable());
+        baseDir = CommonFSUtils.getTableDir(MobUtils.getMobHome(conf), regionInfo.getTable());
       }
       HRegionFileSystem regionFs = HRegionFileSystem.openRegionFromFileSystem(conf, rootFs,
         baseDir, regionInfo, true);
@@ -337,7 +338,7 @@ public final class SnapshotManifest {
   }
 
   private List<StoreFileInfo> getStoreFiles(Path storeDir) throws IOException {
-    FileStatus[] stats = FSUtils.listStatus(rootFs, storeDir);
+    FileStatus[] stats = CommonFSUtils.listStatus(rootFs, storeDir);
     if (stats == null) return null;
 
     ArrayList<StoreFileInfo> storeFiles = new ArrayList<>(stats.length);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifestV1.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifestV1.java
index 742ac2e..b1eca35 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifestV1.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifestV1.java
@@ -27,7 +27,6 @@ import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Executor;
 import java.util.concurrent.ExecutorCompletionService;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -36,11 +35,14 @@ import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
@@ -123,7 +125,8 @@ public final class SnapshotManifestV1 {
   static List<SnapshotRegionManifest> loadRegionManifests(final Configuration conf,
       final Executor executor,final FileSystem fs, final Path snapshotDir,
       final SnapshotDescription desc) throws IOException {
-    FileStatus[] regions = FSUtils.listStatus(fs, snapshotDir, new FSUtils.RegionDirFilter(fs));
+    FileStatus[] regions =
+      CommonFSUtils.listStatus(fs, snapshotDir, new FSUtils.RegionDirFilter(fs));
     if (regions == null) {
       LOG.debug("No regions under directory:" + snapshotDir);
       return null;
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifestV2.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifestV2.java
index f85dcfd..4f3df2f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifestV2.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifestV2.java
@@ -26,7 +26,6 @@ import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Executor;
 import java.util.concurrent.ExecutorCompletionService;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -36,13 +35,15 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.com.google.protobuf.CodedInputStream;
 import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
 import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
@@ -141,7 +142,7 @@ public final class SnapshotManifestV2 {
   static List<SnapshotRegionManifest> loadRegionManifests(final Configuration conf,
       final Executor executor, final FileSystem fs, final Path snapshotDir,
       final SnapshotDescription desc, final int manifestSizeLimit) throws IOException {
-    FileStatus[] manifestFiles = FSUtils.listStatus(fs, snapshotDir, new PathFilter() {
+    FileStatus[] manifestFiles = CommonFSUtils.listStatus(fs, snapshotDir, new PathFilter() {
       @Override
       public boolean accept(Path path) {
         return path.getName().startsWith(SNAPSHOT_MANIFEST_PREFIX);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/HFileContentValidator.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/HFileContentValidator.java
index dd71148..3120d8a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/HFileContentValidator.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/HFileContentValidator.java
@@ -24,13 +24,13 @@ import java.util.List;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.util.AbstractHBaseTool;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.util.hbck.HFileCorruptionChecker;
@@ -53,7 +53,7 @@ public class HFileContentValidator extends AbstractHBaseTool {
    * @throws IOException if a remote or network exception occurs
    */
   private boolean validateHFileContent(Configuration conf) throws IOException {
-    FileSystem fileSystem = FSUtils.getCurrentFileSystem(conf);
+    FileSystem fileSystem = CommonFSUtils.getCurrentFileSystem(conf);
 
     ExecutorService threadPool = createThreadPool(conf);
     HFileCorruptionChecker checker;
@@ -61,7 +61,7 @@ public class HFileContentValidator extends AbstractHBaseTool {
     try {
       checker = new HFileCorruptionChecker(conf, threadPool, false);
 
-      Path rootDir = FSUtils.getRootDir(conf);
+      Path rootDir = CommonFSUtils.getRootDir(conf);
       LOG.info("Validating HFile contents under {}", rootDir);
 
       Collection<Path> tableDirs = FSUtils.getTableDirs(fileSystem, rootDir);
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 a4bb00d..7b306f4 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
@@ -92,7 +92,7 @@ import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
 import org.apache.hadoop.hbase.security.UserProvider;
 import org.apache.hadoop.hbase.security.token.FsDelegationToken;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSHDFSUtils;
+import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.FSVisitor;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.util.Tool;
@@ -566,7 +566,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
             // Check to see if the source and target filesystems are the same
             // If they are the same filesystem, we will try move the files back
             // because previously we moved them to the staging directory.
-            if (FSHDFSUtils.isSameHdfs(getConf(), sourceFs, targetFs)) {
+            if (FSUtils.isSameHdfs(getConf(), sourceFs, targetFs)) {
               for (Pair<byte[], String> el : famPaths) {
                 Path hfileStagingPath = null;
                 Path hfileOrigPath = new Path(el.getSecond());
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSHDFSUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSHDFSUtils.java
deleted file mode 100644
index 301d158..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSHDFSUtils.java
+++ /dev/null
@@ -1,332 +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.util;
-
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.lang.reflect.Method;
-import java.net.InetSocketAddress;
-import java.net.URI;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.DistributedFileSystem;
-import org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException;
-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.collect.Sets;
-
-/**
- * Implementation for hdfs
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class FSHDFSUtils extends FSUtils {
-  private static final Logger LOG = LoggerFactory.getLogger(FSHDFSUtils.class);
-  private static Class dfsUtilClazz;
-  private static Method getNNAddressesMethod;
-
-  /**
-   * @param fs
-   * @param conf
-   * @return A set containing all namenode addresses of fs
-   */
-  private static Set<InetSocketAddress> getNNAddresses(DistributedFileSystem fs,
-                                                      Configuration conf) {
-    Set<InetSocketAddress> addresses = new HashSet<>();
-    String serviceName = fs.getCanonicalServiceName();
-
-    if (serviceName.startsWith("ha-hdfs")) {
-      try {
-        if (dfsUtilClazz == null) {
-          dfsUtilClazz = Class.forName("org.apache.hadoop.hdfs.DFSUtil");
-        }
-        if (getNNAddressesMethod == null) {
-          try {
-            // getNNServiceRpcAddressesForCluster is available only in version
-            // equal to or later than Hadoop 2.6
-            getNNAddressesMethod =
-                dfsUtilClazz.getMethod("getNNServiceRpcAddressesForCluster", Configuration.class);
-          } catch (NoSuchMethodException e) {
-            // If hadoop version is older than hadoop 2.6
-            getNNAddressesMethod =
-                dfsUtilClazz.getMethod("getNNServiceRpcAddresses", Configuration.class);
-          }
-
-        }
-
-        Map<String, Map<String, InetSocketAddress>> addressMap =
-                (Map<String, Map<String, InetSocketAddress>>) getNNAddressesMethod
-                        .invoke(null, conf);
-        String nameService = serviceName.substring(serviceName.indexOf(":") + 1);
-        if (addressMap.containsKey(nameService)) {
-          Map<String, InetSocketAddress> nnMap = addressMap.get(nameService);
-          for (Map.Entry<String, InetSocketAddress> e2 : nnMap.entrySet()) {
-            InetSocketAddress addr = e2.getValue();
-            addresses.add(addr);
-          }
-        }
-      } catch (Exception e) {
-        LOG.warn("DFSUtil.getNNServiceRpcAddresses failed. serviceName=" + serviceName, e);
-      }
-    } else {
-      URI uri = fs.getUri();
-      int port = uri.getPort();
-      if (port < 0) {
-        int idx = serviceName.indexOf(':');
-        port = Integer.parseInt(serviceName.substring(idx+1));
-      }
-      InetSocketAddress addr = new InetSocketAddress(uri.getHost(), port);
-      addresses.add(addr);
-    }
-
-    return addresses;
-  }
-
-  /**
-   * @param conf the Configuration of HBase
-   * @param srcFs
-   * @param desFs
-   * @return Whether srcFs and desFs are on same hdfs or not
-   */
-  public static boolean isSameHdfs(Configuration conf, FileSystem srcFs, FileSystem desFs) {
-    // By getCanonicalServiceName, we could make sure both srcFs and desFs
-    // show a unified format which contains scheme, host and port.
-    String srcServiceName = srcFs.getCanonicalServiceName();
-    String desServiceName = desFs.getCanonicalServiceName();
-
-    if (srcServiceName == null || desServiceName == null) {
-      return false;
-    }
-    if (srcServiceName.equals(desServiceName)) {
-      return true;
-    }
-    if (srcServiceName.startsWith("ha-hdfs") && desServiceName.startsWith("ha-hdfs")) {
-      Collection<String> internalNameServices =
-          conf.getTrimmedStringCollection("dfs.internal.nameservices");
-      if (!internalNameServices.isEmpty()) {
-        if (internalNameServices.contains(srcServiceName.split(":")[1])) {
-          return true;
-        } else {
-          return false;
-        }
-      }
-    }
-    if (srcFs instanceof DistributedFileSystem && desFs instanceof DistributedFileSystem) {
-      //If one serviceName is an HA format while the other is a non-HA format,
-      // maybe they refer to the same FileSystem.
-      //For example, srcFs is "ha-hdfs://nameservices" and desFs is "hdfs://activeNamenode:port"
-      Set<InetSocketAddress> srcAddrs = getNNAddresses((DistributedFileSystem) srcFs, conf);
-      Set<InetSocketAddress> desAddrs = getNNAddresses((DistributedFileSystem) desFs, conf);
-      if (Sets.intersection(srcAddrs, desAddrs).size() > 0) {
-        return true;
-      }
-    }
-
-    return false;
-  }
-
-  /**
-   * Recover the lease from HDFS, retrying multiple times.
-   */
-  @Override
-  public void recoverFileLease(final FileSystem fs, final Path p,
-      Configuration conf, CancelableProgressable reporter)
-  throws IOException {
-    // lease recovery not needed for local file system case.
-    if (!(fs instanceof DistributedFileSystem)) return;
-    recoverDFSFileLease((DistributedFileSystem)fs, p, conf, reporter);
-  }
-
-  /*
-   * Run the dfs recover lease. recoverLease is asynchronous. It returns:
-   *    -false when it starts the lease recovery (i.e. lease recovery not *yet* done)
-   *    - true when the lease recovery has succeeded or the file is closed.
-   * But, we have to be careful.  Each time we call recoverLease, it starts the recover lease
-   * process over from the beginning.  We could put ourselves in a situation where we are
-   * doing nothing but starting a recovery, interrupting it to start again, and so on.
-   * The findings over in HBASE-8354 have it that the namenode will try to recover the lease
-   * on the file's primary node.  If all is well, it should return near immediately.  But,
-   * as is common, it is the very primary node that has crashed and so the namenode will be
-   * stuck waiting on a socket timeout before it will ask another datanode to start the
-   * recovery. It does not help if we call recoverLease in the meantime and in particular,
-   * subsequent to the socket timeout, a recoverLease invocation will cause us to start
-   * over from square one (possibly waiting on socket timeout against primary node).  So,
-   * in the below, we do the following:
-   * 1. Call recoverLease.
-   * 2. If it returns true, break.
-   * 3. If it returns false, wait a few seconds and then call it again.
-   * 4. If it returns true, break.
-   * 5. If it returns false, wait for what we think the datanode socket timeout is
-   * (configurable) and then try again.
-   * 6. If it returns true, break.
-   * 7. If it returns false, repeat starting at step 5. above.
-   *
-   * If HDFS-4525 is available, call it every second and we might be able to exit early.
-   */
-  boolean recoverDFSFileLease(final DistributedFileSystem dfs, final Path p,
-      final Configuration conf, final CancelableProgressable reporter)
-  throws IOException {
-    LOG.info("Recover lease on dfs file " + p);
-    long startWaiting = EnvironmentEdgeManager.currentTime();
-    // Default is 15 minutes. It's huge, but the idea is that if we have a major issue, HDFS
-    // usually needs 10 minutes before marking the nodes as dead. So we're putting ourselves
-    // beyond that limit 'to be safe'.
-    long recoveryTimeout = conf.getInt("hbase.lease.recovery.timeout", 900000) + startWaiting;
-    // This setting should be a little bit above what the cluster dfs heartbeat is set to.
-    long firstPause = conf.getInt("hbase.lease.recovery.first.pause", 4000);
-    // This should be set to how long it'll take for us to timeout against primary datanode if it
-    // is dead.  We set it to 64 seconds, 4 second than the default READ_TIMEOUT in HDFS, the
-    // default value for DFS_CLIENT_SOCKET_TIMEOUT_KEY. If recovery is still failing after this
-    // timeout, then further recovery will take liner backoff with this base, to avoid endless
-    // preemptions when this value is not properly configured.
-    long subsequentPauseBase = conf.getLong("hbase.lease.recovery.dfs.timeout", 64 * 1000);
-
-    Method isFileClosedMeth = null;
-    // whether we need to look for isFileClosed method
-    boolean findIsFileClosedMeth = true;
-    boolean recovered = false;
-    // We break the loop if we succeed the lease recovery, timeout, or we throw an exception.
-    for (int nbAttempt = 0; !recovered; nbAttempt++) {
-      recovered = recoverLease(dfs, nbAttempt, p, startWaiting);
-      if (recovered) break;
-      checkIfCancelled(reporter);
-      if (checkIfTimedout(conf, recoveryTimeout, nbAttempt, p, startWaiting)) break;
-      try {
-        // On the first time through wait the short 'firstPause'.
-        if (nbAttempt == 0) {
-          Thread.sleep(firstPause);
-        } else {
-          // Cycle here until (subsequentPause * nbAttempt) elapses.  While spinning, check
-          // isFileClosed if available (should be in hadoop 2.0.5... not in hadoop 1 though.
-          long localStartWaiting = EnvironmentEdgeManager.currentTime();
-          while ((EnvironmentEdgeManager.currentTime() - localStartWaiting) <
-              subsequentPauseBase * nbAttempt) {
-            Thread.sleep(conf.getInt("hbase.lease.recovery.pause", 1000));
-            if (findIsFileClosedMeth) {
-              try {
-                isFileClosedMeth = dfs.getClass().getMethod("isFileClosed",
-                  new Class[]{ Path.class });
-              } catch (NoSuchMethodException nsme) {
-                LOG.debug("isFileClosed not available");
-              } finally {
-                findIsFileClosedMeth = false;
-              }
-            }
-            if (isFileClosedMeth != null && isFileClosed(dfs, isFileClosedMeth, p)) {
-              recovered = true;
-              break;
-            }
-            checkIfCancelled(reporter);
-          }
-        }
-      } catch (InterruptedException ie) {
-        InterruptedIOException iioe = new InterruptedIOException();
-        iioe.initCause(ie);
-        throw iioe;
-      }
-    }
-    return recovered;
-  }
-
-  boolean checkIfTimedout(final Configuration conf, final long recoveryTimeout,
-      final int nbAttempt, final Path p, final long startWaiting) {
-    if (recoveryTimeout < EnvironmentEdgeManager.currentTime()) {
-      LOG.warn("Cannot recoverLease after trying for " +
-        conf.getInt("hbase.lease.recovery.timeout", 900000) +
-        "ms (hbase.lease.recovery.timeout); continuing, but may be DATALOSS!!!; " +
-        getLogMessageDetail(nbAttempt, p, startWaiting));
-      return true;
-    }
-    return false;
-  }
-
-  /**
-   * Try to recover the lease.
-   * @param dfs
-   * @param nbAttempt
-   * @param p
-   * @param startWaiting
-   * @return True if dfs#recoverLease came by true.
-   * @throws FileNotFoundException
-   */
-  boolean recoverLease(final DistributedFileSystem dfs, final int nbAttempt, final Path p,
-      final long startWaiting)
-  throws FileNotFoundException {
-    boolean recovered = false;
-    try {
-      recovered = dfs.recoverLease(p);
-      LOG.info((recovered? "Recovered lease, ": "Failed to recover lease, ") +
-        getLogMessageDetail(nbAttempt, p, startWaiting));
-    } catch (IOException e) {
-      if (e instanceof LeaseExpiredException && e.getMessage().contains("File does not exist")) {
-        // This exception comes out instead of FNFE, fix it
-        throw new FileNotFoundException("The given WAL wasn't found at " + p);
-      } else if (e instanceof FileNotFoundException) {
-        throw (FileNotFoundException)e;
-      }
-      LOG.warn(getLogMessageDetail(nbAttempt, p, startWaiting), e);
-    }
-    return recovered;
-  }
-
-  /**
-   * @param nbAttempt
-   * @param p
-   * @param startWaiting
-   * @return Detail to append to any log message around lease recovering.
-   */
-  private String getLogMessageDetail(final int nbAttempt, final Path p, final long startWaiting) {
-    return "attempt=" + nbAttempt + " on file=" + p + " after " +
-      (EnvironmentEdgeManager.currentTime() - startWaiting) + "ms";
-  }
-
-  /**
-   * Call HDFS-4525 isFileClosed if it is available.
-   * @param dfs
-   * @param m
-   * @param p
-   * @return True if file is closed.
-   */
-  private boolean isFileClosed(final DistributedFileSystem dfs, final Method m, final Path p) {
-    try {
-      return (Boolean) m.invoke(dfs, p);
-    } catch (SecurityException e) {
-      LOG.warn("No access", e);
-    } catch (Exception e) {
-      LOG.warn("Failed invocation for " + p.toString(), e);
-    }
-    return false;
-  }
-
-  void checkIfCancelled(final CancelableProgressable reporter)
-  throws InterruptedIOException {
-    if (reporter == null) return;
-    if (!reporter.progress()) throw new InterruptedIOException("Operation cancelled");
-  }
-}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java
index 467d66f..2329a0d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java
@@ -104,7 +104,7 @@ public class FSTableDescriptors implements TableDescriptors {
    * filesystem where that root dir lives. This instance can do write operations (is not read only).
    */
   public FSTableDescriptors(final Configuration conf) throws IOException {
-    this(FSUtils.getCurrentFileSystem(conf), FSUtils.getRootDir(conf));
+    this(CommonFSUtils.getCurrentFileSystem(conf), CommonFSUtils.getRootDir(conf));
   }
 
   public FSTableDescriptors(final FileSystem fs, final Path rootdir) {
@@ -120,8 +120,8 @@ public class FSTableDescriptors implements TableDescriptors {
   }
 
   public static void tryUpdateMetaTableDescriptor(Configuration conf) throws IOException {
-    tryUpdateMetaTableDescriptor(conf, FSUtils.getCurrentFileSystem(conf), FSUtils.getRootDir(conf),
-      null);
+    tryUpdateMetaTableDescriptor(conf, CommonFSUtils.getCurrentFileSystem(conf),
+      CommonFSUtils.getRootDir(conf), null);
   }
 
   public static void tryUpdateMetaTableDescriptor(Configuration conf, FileSystem fs, Path rootdir,
@@ -137,7 +137,7 @@ public class FSTableDescriptors implements TableDescriptors {
       TableDescriptor td = builder.build();
       LOG.info("Creating new hbase:meta table descriptor {}", td);
       TableName tableName = td.getTableName();
-      Path tableDir = FSUtils.getTableDir(rootdir, tableName);
+      Path tableDir = CommonFSUtils.getTableDir(rootdir, tableName);
       Path p = writeTableDescriptor(fs, td, tableDir, getTableInfoPath(fs, tableDir, true));
       if (p == null) {
         throw new IOException("Failed update hbase:meta table descriptor");
@@ -258,7 +258,7 @@ public class FSTableDescriptors implements TableDescriptors {
       for (Path d : FSUtils.getTableDirs(fs, rootdir)) {
         TableDescriptor htd = null;
         try {
-          htd = get(FSUtils.getTableName(d));
+          htd = get(CommonFSUtils.getTableName(d));
         } catch (FileNotFoundException fnfe) {
           // inability of retrieving one HTD shouldn't stop getting the remaining
           LOG.warn("Trouble retrieving htd", fnfe);
@@ -284,17 +284,17 @@ public class FSTableDescriptors implements TableDescriptors {
   throws IOException {
     Map<String, TableDescriptor> htds = new TreeMap<>();
     List<Path> tableDirs =
-        FSUtils.getLocalTableDirs(fs, FSUtils.getNamespaceDir(rootdir, name));
+        FSUtils.getLocalTableDirs(fs, CommonFSUtils.getNamespaceDir(rootdir, name));
     for (Path d: tableDirs) {
       TableDescriptor htd = null;
       try {
-        htd = get(FSUtils.getTableName(d));
+        htd = get(CommonFSUtils.getTableName(d));
       } catch (FileNotFoundException fnfe) {
         // inability of retrieving one HTD shouldn't stop getting the remaining
         LOG.warn("Trouble retrieving htd", fnfe);
       }
       if (htd == null) continue;
-      htds.put(FSUtils.getTableName(d).getNameAsString(), htd);
+      htds.put(CommonFSUtils.getTableName(d).getNameAsString(), htd);
     }
     return htds;
   }
@@ -383,8 +383,8 @@ public class FSTableDescriptors implements TableDescriptors {
    */
   // only visible for FSTableDescriptorMigrationToSubdir, can be removed with that
   static FileStatus getCurrentTableInfoStatus(FileSystem fs, Path dir, boolean removeOldFiles)
-  throws IOException {
-    FileStatus [] status = FSUtils.listStatus(fs, dir, TABLEINFO_PATHFILTER);
+    throws IOException {
+    FileStatus[] status = CommonFSUtils.listStatus(fs, dir, TABLEINFO_PATHFILTER);
     if (status == null || status.length < 1) return null;
     FileStatus mostCurrent = null;
     for (FileStatus file : status) {
@@ -423,8 +423,9 @@ public class FSTableDescriptors implements TableDescriptors {
   /**
    * Return the table directory in HDFS
    */
-  @VisibleForTesting Path getTableDir(final TableName tableName) {
-    return FSUtils.getTableDir(rootdir, tableName);
+  @VisibleForTesting
+  Path getTableDir(final TableName tableName) {
+    return CommonFSUtils.getTableDir(rootdir, tableName);
   }
 
   private static final PathFilter TABLEINFO_PATHFILTER = new PathFilter() {
@@ -490,7 +491,7 @@ public class FSTableDescriptors implements TableDescriptors {
    */
   public static TableDescriptor getTableDescriptorFromFs(FileSystem fs,
       Path hbaseRootDir, TableName tableName) throws IOException {
-    Path tableDir = FSUtils.getTableDir(hbaseRootDir, tableName);
+    Path tableDir = CommonFSUtils.getTableDir(hbaseRootDir, tableName);
     return getTableDescriptorFromFs(fs, tableDir);
   }
 
@@ -556,12 +557,12 @@ public class FSTableDescriptors implements TableDescriptors {
    */
   private static void deleteTableDescriptorFiles(FileSystem fs, Path dir, int maxSequenceId)
   throws IOException {
-    FileStatus [] status = FSUtils.listStatus(fs, dir, TABLEINFO_PATHFILTER);
+    FileStatus [] status = CommonFSUtils.listStatus(fs, dir, TABLEINFO_PATHFILTER);
     for (FileStatus file : status) {
       Path path = file.getPath();
       int sequenceId = getTableInfoSequenceId(path);
       if (sequenceId <= maxSequenceId) {
-        boolean success = FSUtils.delete(fs, path, false);
+        boolean success = CommonFSUtils.delete(fs, path, false);
         if (success) {
           LOG.debug("Deleted " + path);
         } else {
@@ -621,7 +622,7 @@ public class FSTableDescriptors implements TableDescriptors {
       } catch (IOException ioe) {
         // Presume clash of names or something; go around again.
         LOG.debug("Failed write and/or rename; retrying", ioe);
-        if (!FSUtils.deleteDirectory(fs, tempPath)) {
+        if (!CommonFSUtils.deleteDirectory(fs, tempPath)) {
           LOG.warn("Failed cleanup of " + tempPath);
         }
         tableInfoDirPath = null;
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
index 2c85727..ece76fb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
@@ -28,14 +28,18 @@ import java.io.InterruptedIOException;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 import java.net.InetSocketAddress;
+import java.net.URI;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Locale;
 import java.util.Map;
+import java.util.Set;
 import java.util.Vector;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ExecutionException;
@@ -46,7 +50,6 @@ import java.util.concurrent.FutureTask;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import java.util.regex.Pattern;
-
 import org.apache.commons.lang3.ArrayUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
@@ -55,6 +58,7 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.FilterFileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.fs.permission.FsPermission;
@@ -62,8 +66,8 @@ import org.apache.hadoop.hbase.ClusterId;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HDFSBlocksDistribution;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
@@ -73,12 +77,13 @@ import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSHedgedReadMetrics;
+import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.util.Progressable;
-import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
@@ -87,6 +92,7 @@ import org.slf4j.LoggerFactory;
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hbase.thirdparty.com.google.common.base.Throwables;
 import org.apache.hbase.thirdparty.com.google.common.collect.Iterators;
+import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
 import org.apache.hbase.thirdparty.com.google.common.primitives.Ints;
 
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
@@ -96,7 +102,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos;
  * Utility methods for interacting with the underlying file system.
  */
 @InterfaceAudience.Private
-public abstract class FSUtils extends CommonFSUtils {
+public final class FSUtils {
   private static final Logger LOG = LoggerFactory.getLogger(FSUtils.class);
 
   private static final String THREAD_POOLSIZE = "hbase.client.localityCheck.threadPoolSize";
@@ -106,8 +112,7 @@ public abstract class FSUtils extends CommonFSUtils {
   @VisibleForTesting // currently only used in testing. TODO refactor into a test class
   public static final boolean WINDOWS = System.getProperty("os.name").startsWith("Windows");
 
-  protected FSUtils() {
-    super();
+  private FSUtils() {
   }
 
   /**
@@ -160,32 +165,17 @@ public abstract class FSUtils extends CommonFSUtils {
     return result;
   }
 
-  public static FSUtils getInstance(FileSystem fs, Configuration conf) {
-    String scheme = fs.getUri().getScheme();
-    if (scheme == null) {
-      LOG.warn("Could not find scheme for uri " +
-          fs.getUri() + ", default to hdfs");
-      scheme = "hdfs";
-    }
-    Class<?> fsUtilsClass = conf.getClass("hbase.fsutil." +
-        scheme + ".impl", FSHDFSUtils.class); // Default to HDFS impl
-    FSUtils fsUtils = (FSUtils)ReflectionUtils.newInstance(fsUtilsClass, conf);
-    return fsUtils;
-  }
-
   /**
    * Delete the region directory if exists.
-   * @param conf
-   * @param hri
    * @return True if deleted the region directory.
    * @throws IOException
    */
-  public static boolean deleteRegionDir(final Configuration conf, final HRegionInfo hri)
-  throws IOException {
-    Path rootDir = getRootDir(conf);
+  public static boolean deleteRegionDir(final Configuration conf, final RegionInfo hri)
+    throws IOException {
+    Path rootDir = CommonFSUtils.getRootDir(conf);
     FileSystem fs = rootDir.getFileSystem(conf);
-    return deleteDirectory(fs,
-      new Path(getTableDir(rootDir, hri.getTable()), hri.getEncodedName()));
+    return CommonFSUtils.deleteDirectory(fs,
+      new Path(CommonFSUtils.getTableDir(rootDir, hri.getTable()), hri.getEncodedName()));
   }
 
  /**
@@ -195,7 +185,7 @@ public abstract class FSUtils extends CommonFSUtils {
    * <li>apply the umask in the configuration (if it is enabled)</li>
    * <li>use the fs configured buffer size (or 4096 if not set)</li>
    * <li>use the configured column family replication or default replication if
-   * {@link HColumnDescriptor#DEFAULT_DFS_REPLICATION}</li>
+   * {@link ColumnFamilyDescriptorBuilder#DEFAULT_DFS_REPLICATION}</li>
    * <li>use the default block size</li>
    * <li>not track progress</li>
    * </ol>
@@ -203,38 +193,38 @@ public abstract class FSUtils extends CommonFSUtils {
    * @param fs {@link FileSystem} on which to write the file
    * @param path {@link Path} to the file to write
    * @param perm permissions
-   * @param favoredNodes
+   * @param favoredNodes favored data nodes
    * @return output stream to the created file
    * @throws IOException if the file cannot be created
    */
   public static FSDataOutputStream create(Configuration conf, FileSystem fs, Path path,
-      FsPermission perm, InetSocketAddress[] favoredNodes) throws IOException {
+    FsPermission perm, InetSocketAddress[] favoredNodes) throws IOException {
     if (fs instanceof HFileSystem) {
-      FileSystem backingFs = ((HFileSystem)fs).getBackingFs();
+      FileSystem backingFs = ((HFileSystem) fs).getBackingFs();
       if (backingFs instanceof DistributedFileSystem) {
         // Try to use the favoredNodes version via reflection to allow backwards-
         // compatibility.
-        short replication = Short.parseShort(conf.get(HColumnDescriptor.DFS_REPLICATION,
-          String.valueOf(HColumnDescriptor.DEFAULT_DFS_REPLICATION)));
+        short replication = Short.parseShort(conf.get(ColumnFamilyDescriptorBuilder.DFS_REPLICATION,
+          String.valueOf(ColumnFamilyDescriptorBuilder.DEFAULT_DFS_REPLICATION)));
         try {
-          return (FSDataOutputStream) (DistributedFileSystem.class.getDeclaredMethod("create",
-            Path.class, FsPermission.class, boolean.class, int.class, short.class, long.class,
-            Progressable.class, InetSocketAddress[].class).invoke(backingFs, path, perm, true,
-            getDefaultBufferSize(backingFs),
-            replication > 0 ? replication : getDefaultReplication(backingFs, path),
-            getDefaultBlockSize(backingFs, path), null, favoredNodes));
+          return (FSDataOutputStream) (DistributedFileSystem.class
+            .getDeclaredMethod("create", Path.class, FsPermission.class, boolean.class, int.class,
+              short.class, long.class, Progressable.class, InetSocketAddress[].class)
+            .invoke(backingFs, path, perm, true, CommonFSUtils.getDefaultBufferSize(backingFs),
+              replication > 0 ? replication : CommonFSUtils.getDefaultReplication(backingFs, path),
+              CommonFSUtils.getDefaultBlockSize(backingFs, path), null, favoredNodes));
         } catch (InvocationTargetException ite) {
           // Function was properly called, but threw it's own exception.
           throw new IOException(ite.getCause());
         } catch (NoSuchMethodException e) {
           LOG.debug("DFS Client does not support most favored nodes create; using default create");
           LOG.trace("Ignoring; use default create", e);
-        } catch (IllegalArgumentException | SecurityException |  IllegalAccessException e) {
+        } catch (IllegalArgumentException | SecurityException | IllegalAccessException e) {
           LOG.debug("Ignoring (most likely Reflection related exception) " + e);
         }
       }
     }
-    return create(fs, path, perm, true);
+    return CommonFSUtils.create(fs, path, perm, true);
   }
 
   /**
@@ -779,10 +769,9 @@ public abstract class FSUtils extends CommonFSUtils {
    *
    * @throws IOException When scanning the directory fails.
    */
-  public static Map<String, Integer> getTableFragmentation(
-    final HMaster master)
-  throws IOException {
-    Path path = getRootDir(master.getConfiguration());
+  public static Map<String, Integer> getTableFragmentation(final HMaster master)
+    throws IOException {
+    Path path = CommonFSUtils.getRootDir(master.getConfiguration());
     // since HMaster.getFileSystem() is package private
     FileSystem fs = path.getFileSystem(master.getConfiguration());
     return getTableFragmentation(fs, path);
@@ -828,7 +817,7 @@ public abstract class FSUtils extends CommonFSUtils {
         }
       }
       // compute percentage per table and store in result list
-      frags.put(FSUtils.getTableName(d).getNameAsString(),
+      frags.put(CommonFSUtils.getTableName(d).getNameAsString(),
         cfCount == 0? 0: Math.round((float) cfFrag / cfCount * 100));
     }
     // set overall percentage for all tables
@@ -933,17 +922,6 @@ public abstract class FSUtils extends CommonFSUtils {
     }
   }
 
-  /**
-   * Recover file lease. Used when a file might be suspect
-   * to be had been left open by another process.
-   * @param fs FileSystem handle
-   * @param p Path of file to recover lease
-   * @param conf Configuration handle
-   * @throws IOException
-   */
-  public abstract void recoverFileLease(final FileSystem fs, final Path p,
-      Configuration conf, CancelableProgressable reporter) throws IOException;
-
   public static List<Path> getTableDirs(final FileSystem fs, final Path rootdir)
       throws IOException {
     List<Path> tableDirs = new ArrayList<>();
@@ -1024,7 +1002,7 @@ public abstract class FSUtils extends CommonFSUtils {
   }
 
   public static Path getRegionDirFromRootDir(Path rootDir, RegionInfo region) {
-    return getRegionDirFromTableDir(getTableDir(rootDir, region.getTable()), region);
+    return getRegionDirFromTableDir(CommonFSUtils.getTableDir(rootDir, region.getTable()), region);
   }
 
   public static Path getRegionDirFromTableDir(Path tableDir, RegionInfo region) {
@@ -1267,7 +1245,7 @@ public abstract class FSUtils extends CommonFSUtils {
         resultMap == null ? new ConcurrentHashMap<>(128, 0.75f, 32) : resultMap;
 
     // only include the directory paths to tables
-    Path tableDir = FSUtils.getTableDir(hbaseRootDir, tableName);
+    Path tableDir = CommonFSUtils.getTableDir(hbaseRootDir, tableName);
     // Inside a table, there are compaction.dir directories to skip.  Otherwise, all else
     // should be regions.
     final FamilyDirFilter familyFilter = new FamilyDirFilter(fs);
@@ -1365,8 +1343,8 @@ public abstract class FSUtils extends CommonFSUtils {
       if (!exceptions.isEmpty()) {
         // Just throw the first exception as an indication something bad happened
         // Don't need to propagate all the exceptions, we already logged them all anyway
-        Throwables.propagateIfInstanceOf(exceptions.firstElement(), IOException.class);
-        throw Throwables.propagate(exceptions.firstElement());
+        Throwables.propagateIfPossible(exceptions.firstElement(), IOException.class);
+        throw new IOException(exceptions.firstElement());
       }
     }
 
@@ -1467,8 +1445,8 @@ public abstract class FSUtils extends CommonFSUtils {
 
     // only include the directory paths to tables
     for (Path tableDir : FSUtils.getTableDirs(fs, hbaseRootDir)) {
-      getTableStoreFilePathMap(map, fs, hbaseRootDir,
-          FSUtils.getTableName(tableDir), sfFilter, executor, progressReporter);
+      getTableStoreFilePathMap(map, fs, hbaseRootDir, CommonFSUtils.getTableName(tableDir),
+        sfFilter, executor, progressReporter);
     }
     return map;
   }
@@ -1607,17 +1585,19 @@ public abstract class FSUtils extends CommonFSUtils {
    *           in case of file system errors or interrupts
    */
   private static void getRegionLocalityMappingFromFS(final Configuration conf,
-      final String desiredTable, int threadPoolSize,
-      final Map<String, Map<String, Float>> regionDegreeLocalityMapping) throws IOException {
-    final FileSystem fs =  FileSystem.get(conf);
-    final Path rootPath = FSUtils.getRootDir(conf);
+    final String desiredTable, int threadPoolSize,
+    final Map<String, Map<String, Float>> regionDegreeLocalityMapping) throws IOException {
+    final FileSystem fs = FileSystem.get(conf);
+    final Path rootPath = CommonFSUtils.getRootDir(conf);
     final long startTime = EnvironmentEdgeManager.currentTime();
     final Path queryPath;
     // The table files are in ${hbase.rootdir}/data/<namespace>/<table>/*
     if (null == desiredTable) {
-      queryPath = new Path(new Path(rootPath, HConstants.BASE_NAMESPACE_DIR).toString() + "/*/*/*/");
+      queryPath =
+        new Path(new Path(rootPath, HConstants.BASE_NAMESPACE_DIR).toString() + "/*/*/*/");
     } else {
-      queryPath = new Path(FSUtils.getTableDir(rootPath, TableName.valueOf(desiredTable)).toString() + "/*/");
+      queryPath = new Path(
+        CommonFSUtils.getTableDir(rootPath, TableName.valueOf(desiredTable)).toString() + "/*/");
     }
 
     // reject all paths that are not appropriate
@@ -1743,7 +1723,9 @@ public abstract class FSUtils extends CommonFSUtils {
    */
   public static DFSHedgedReadMetrics getDFSHedgedReadMetrics(final Configuration c)
       throws IOException {
-    if (!isHDFS(c)) return null;
+    if (!CommonFSUtils.isHDFS(c)) {
+      return null;
+    }
     // getHedgedReadMetrics is package private. Get the DFSClient instance that is internal
     // to the DFS FS instance and make the method getHedgedReadMetrics accessible, then invoke it
     // to get the singleton instance of DFSHedgedReadMetrics shared by DFSClients.
@@ -1812,4 +1794,259 @@ public abstract class FSUtils extends CommonFSUtils {
     }
     return traversedPaths;
   }
+
+  /**
+   * @return A set containing all namenode addresses of fs
+   */
+  private static Set<InetSocketAddress> getNNAddresses(DistributedFileSystem fs,
+    Configuration conf) {
+    Set<InetSocketAddress> addresses = new HashSet<>();
+    String serviceName = fs.getCanonicalServiceName();
+
+    if (serviceName.startsWith("ha-hdfs")) {
+      try {
+        Map<String, Map<String, InetSocketAddress>> addressMap =
+          DFSUtil.getNNServiceRpcAddressesForCluster(conf);
+        String nameService = serviceName.substring(serviceName.indexOf(":") + 1);
+        if (addressMap.containsKey(nameService)) {
+          Map<String, InetSocketAddress> nnMap = addressMap.get(nameService);
+          for (Map.Entry<String, InetSocketAddress> e2 : nnMap.entrySet()) {
+            InetSocketAddress addr = e2.getValue();
+            addresses.add(addr);
+          }
+        }
+      } catch (Exception e) {
+        LOG.warn("DFSUtil.getNNServiceRpcAddresses failed. serviceName=" + serviceName, e);
+      }
+    } else {
+      URI uri = fs.getUri();
+      int port = uri.getPort();
+      if (port < 0) {
+        int idx = serviceName.indexOf(':');
+        port = Integer.parseInt(serviceName.substring(idx + 1));
+      }
+      InetSocketAddress addr = new InetSocketAddress(uri.getHost(), port);
+      addresses.add(addr);
+    }
+
+    return addresses;
+  }
+
+  /**
+   * @param conf the Configuration of HBase
+   * @return Whether srcFs and desFs are on same hdfs or not
+   */
+  public static boolean isSameHdfs(Configuration conf, FileSystem srcFs, FileSystem desFs) {
+    // By getCanonicalServiceName, we could make sure both srcFs and desFs
+    // show a unified format which contains scheme, host and port.
+    String srcServiceName = srcFs.getCanonicalServiceName();
+    String desServiceName = desFs.getCanonicalServiceName();
+
+    if (srcServiceName == null || desServiceName == null) {
+      return false;
+    }
+    if (srcServiceName.equals(desServiceName)) {
+      return true;
+    }
+    if (srcServiceName.startsWith("ha-hdfs") && desServiceName.startsWith("ha-hdfs")) {
+      Collection<String> internalNameServices =
+        conf.getTrimmedStringCollection("dfs.internal.nameservices");
+      if (!internalNameServices.isEmpty()) {
+        if (internalNameServices.contains(srcServiceName.split(":")[1])) {
+          return true;
+        } else {
+          return false;
+        }
+      }
+    }
+    if (srcFs instanceof DistributedFileSystem && desFs instanceof DistributedFileSystem) {
+      // If one serviceName is an HA format while the other is a non-HA format,
+      // maybe they refer to the same FileSystem.
+      // For example, srcFs is "ha-hdfs://nameservices" and desFs is "hdfs://activeNamenode:port"
+      Set<InetSocketAddress> srcAddrs = getNNAddresses((DistributedFileSystem) srcFs, conf);
+      Set<InetSocketAddress> desAddrs = getNNAddresses((DistributedFileSystem) desFs, conf);
+      if (Sets.intersection(srcAddrs, desAddrs).size() > 0) {
+        return true;
+      }
+    }
+
+    return false;
+  }
+
+  public static void recoverFileLease(FileSystem fs, Path p, Configuration conf)
+    throws IOException {
+    recoverFileLease(fs, p, conf, null);
+  }
+
+  /**
+   * Recover the lease from HDFS, retrying multiple times.
+   */
+  public static void recoverFileLease(FileSystem fs, Path p, Configuration conf,
+    CancelableProgressable reporter) throws IOException {
+    if (fs instanceof FilterFileSystem) {
+      fs = ((FilterFileSystem) fs).getRawFileSystem();
+    }
+    // lease recovery not needed for local file system case.
+    if (!(fs instanceof DistributedFileSystem)) {
+      return;
+    }
+    recoverDFSFileLease((DistributedFileSystem) fs, p, conf, reporter);
+  }
+
+  /*
+   * Run the dfs recover lease. recoverLease is asynchronous. It returns: -false when it starts the
+   * lease recovery (i.e. lease recovery not *yet* done) - true when the lease recovery has
+   * succeeded or the file is closed. But, we have to be careful. Each time we call recoverLease, it
+   * starts the recover lease process over from the beginning. We could put ourselves in a situation
+   * where we are doing nothing but starting a recovery, interrupting it to start again, and so on.
+   * The findings over in HBASE-8354 have it that the namenode will try to recover the lease on the
+   * file's primary node. If all is well, it should return near immediately. But, as is common, it
+   * is the very primary node that has crashed and so the namenode will be stuck waiting on a socket
+   * timeout before it will ask another datanode to start the recovery. It does not help if we call
+   * recoverLease in the meantime and in particular, subsequent to the socket timeout, a
+   * recoverLease invocation will cause us to start over from square one (possibly waiting on socket
+   * timeout against primary node). So, in the below, we do the following: 1. Call recoverLease. 2.
+   * If it returns true, break. 3. If it returns false, wait a few seconds and then call it again.
+   * 4. If it returns true, break. 5. If it returns false, wait for what we think the datanode
+   * socket timeout is (configurable) and then try again. 6. If it returns true, break. 7. If it
+   * returns false, repeat starting at step 5. above. If HDFS-4525 is available, call it every
+   * second and we might be able to exit early.
+   */
+  private static boolean recoverDFSFileLease(final DistributedFileSystem dfs, final Path p,
+    final Configuration conf, final CancelableProgressable reporter) throws IOException {
+    LOG.info("Recover lease on dfs file " + p);
+    long startWaiting = EnvironmentEdgeManager.currentTime();
+    // Default is 15 minutes. It's huge, but the idea is that if we have a major issue, HDFS
+    // usually needs 10 minutes before marking the nodes as dead. So we're putting ourselves
+    // beyond that limit 'to be safe'.
+    long recoveryTimeout = conf.getInt("hbase.lease.recovery.timeout", 900000) + startWaiting;
+    // This setting should be a little bit above what the cluster dfs heartbeat is set to.
+    long firstPause = conf.getInt("hbase.lease.recovery.first.pause", 4000);
+    // This should be set to how long it'll take for us to timeout against primary datanode if it
+    // is dead. We set it to 64 seconds, 4 second than the default READ_TIMEOUT in HDFS, the
+    // default value for DFS_CLIENT_SOCKET_TIMEOUT_KEY. If recovery is still failing after this
+    // timeout, then further recovery will take liner backoff with this base, to avoid endless
+    // preemptions when this value is not properly configured.
+    long subsequentPauseBase = conf.getLong("hbase.lease.recovery.dfs.timeout", 64 * 1000);
+
+    Method isFileClosedMeth = null;
+    // whether we need to look for isFileClosed method
+    boolean findIsFileClosedMeth = true;
+    boolean recovered = false;
+    // We break the loop if we succeed the lease recovery, timeout, or we throw an exception.
+    for (int nbAttempt = 0; !recovered; nbAttempt++) {
+      recovered = recoverLease(dfs, nbAttempt, p, startWaiting);
+      if (recovered) {
+        break;
+      }
+      checkIfCancelled(reporter);
+      if (checkIfTimedout(conf, recoveryTimeout, nbAttempt, p, startWaiting)) {
+        break;
+      }
+      try {
+        // On the first time through wait the short 'firstPause'.
+        if (nbAttempt == 0) {
+          Thread.sleep(firstPause);
+        } else {
+          // Cycle here until (subsequentPause * nbAttempt) elapses. While spinning, check
+          // isFileClosed if available (should be in hadoop 2.0.5... not in hadoop 1 though.
+          long localStartWaiting = EnvironmentEdgeManager.currentTime();
+          while ((EnvironmentEdgeManager.currentTime() - localStartWaiting) < subsequentPauseBase *
+            nbAttempt) {
+            Thread.sleep(conf.getInt("hbase.lease.recovery.pause", 1000));
+            if (findIsFileClosedMeth) {
+              try {
+                isFileClosedMeth =
+                  dfs.getClass().getMethod("isFileClosed", new Class[] { Path.class });
+              } catch (NoSuchMethodException nsme) {
+                LOG.debug("isFileClosed not available");
+              } finally {
+                findIsFileClosedMeth = false;
+              }
+            }
+            if (isFileClosedMeth != null && isFileClosed(dfs, isFileClosedMeth, p)) {
+              recovered = true;
+              break;
+            }
+            checkIfCancelled(reporter);
+          }
+        }
+      } catch (InterruptedException ie) {
+        InterruptedIOException iioe = new InterruptedIOException();
+        iioe.initCause(ie);
+        throw iioe;
+      }
+    }
+    return recovered;
+  }
+
+  private static boolean checkIfTimedout(final Configuration conf, final long recoveryTimeout,
+    final int nbAttempt, final Path p, final long startWaiting) {
+    if (recoveryTimeout < EnvironmentEdgeManager.currentTime()) {
+      LOG.warn("Cannot recoverLease after trying for " +
+        conf.getInt("hbase.lease.recovery.timeout", 900000) +
+        "ms (hbase.lease.recovery.timeout); continuing, but may be DATALOSS!!!; " +
+        getLogMessageDetail(nbAttempt, p, startWaiting));
+      return true;
+    }
+    return false;
+  }
+
+  /**
+   * Try to recover the lease.
+   * @return True if dfs#recoverLease came by true.
+   */
+  private static boolean recoverLease(final DistributedFileSystem dfs, final int nbAttempt,
+    final Path p, final long startWaiting) throws FileNotFoundException {
+    boolean recovered = false;
+    try {
+      recovered = dfs.recoverLease(p);
+      LOG.info((recovered ? "Recovered lease, " : "Failed to recover lease, ") +
+        getLogMessageDetail(nbAttempt, p, startWaiting));
+    } catch (IOException e) {
+      if (e instanceof LeaseExpiredException && e.getMessage().contains("File does not exist")) {
+        // This exception comes out instead of FNFE, fix it
+        throw new FileNotFoundException("The given WAL wasn't found at " + p);
+      } else if (e instanceof FileNotFoundException) {
+        throw (FileNotFoundException) e;
+      }
+      LOG.warn(getLogMessageDetail(nbAttempt, p, startWaiting), e);
+    }
+    return recovered;
+  }
+
+  /**
+   * @return Detail to append to any log message around lease recovering.
+   */
+  private static String getLogMessageDetail(final int nbAttempt, final Path p,
+    final long startWaiting) {
+    return "attempt=" + nbAttempt + " on file=" + p + " after " +
+      (EnvironmentEdgeManager.currentTime() - startWaiting) + "ms";
+  }
+
+  /**
+   * Call HDFS-4525 isFileClosed if it is available.
+   * @return True if file is closed.
+   */
+  private static boolean isFileClosed(final DistributedFileSystem dfs, final Method m,
+    final Path p) {
+    try {
+      return (Boolean) m.invoke(dfs, p);
+    } catch (SecurityException e) {
+      LOG.warn("No access", e);
+    } catch (Exception e) {
+      LOG.warn("Failed invocation for " + p.toString(), e);
+    }
+    return false;
+  }
+
+  private static void checkIfCancelled(final CancelableProgressable reporter)
+    throws InterruptedIOException {
+    if (reporter == null) {
+      return;
+    }
+    if (!reporter.progress()) {
+      throw new InterruptedIOException("Operation cancelled");
+    }
+  }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSVisitor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSVisitor.java
index 24cd223..c3858ae 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSVisitor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSVisitor.java
@@ -20,14 +20,13 @@ package org.apache.hadoop.hbase.util;
 
 import java.io.IOException;
 import java.util.List;
-
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Utility methods for interacting with the hbase.root file system.
@@ -92,7 +91,7 @@ public final class FSVisitor {
       String familyName = familyDir.getName();
 
       // get all the storeFiles in the family
-      FileStatus[] storeFiles = FSUtils.listStatus(fs, familyDir, fileFilter);
+      FileStatus[] storeFiles = CommonFSUtils.listStatus(fs, familyDir, fileFilter);
       if (storeFiles == null) {
         if (LOG.isTraceEnabled()) {
           LOG.trace("No hfiles found for family: " + familyDir + ", skipping.");
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 c347213..1783d7d 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
@@ -401,7 +401,7 @@ public class HBaseFsck extends Configured implements Closeable {
    */
   @VisibleForTesting
   public static Path getTmpDir(Configuration conf) throws IOException {
-    return new Path(FSUtils.getRootDir(conf), HConstants.HBASE_TEMP_DIRECTORY);
+    return new Path(CommonFSUtils.getRootDir(conf), HConstants.HBASE_TEMP_DIRECTORY);
   }
 
   private static class FileLockCallable implements Callable<FSDataOutputStream> {
@@ -424,9 +424,9 @@ public class HBaseFsck extends Configured implements Closeable {
     @Override
     public FSDataOutputStream call() throws IOException {
       try {
-        FileSystem fs = FSUtils.getCurrentFileSystem(this.conf);
-        FsPermission defaultPerms = FSUtils.getFilePermissions(fs, this.conf,
-            HConstants.DATA_FILE_UMASK_KEY);
+        FileSystem fs = CommonFSUtils.getCurrentFileSystem(this.conf);
+        FsPermission defaultPerms =
+          CommonFSUtils.getFilePermissions(fs, this.conf, HConstants.DATA_FILE_UMASK_KEY);
         Path tmpDir = getTmpDir(conf);
         this.hbckLockPath = new Path(tmpDir, HBCK_LOCK_FILE);
         fs.mkdirs(tmpDir);
@@ -453,7 +453,7 @@ public class HBaseFsck extends Configured implements Closeable {
       IOException exception = null;
       do {
         try {
-          return FSUtils.create(fs, hbckLockFilePath, defaultPerms, false);
+          return CommonFSUtils.create(fs, hbckLockFilePath, defaultPerms, false);
         } catch (IOException ioe) {
           LOG.info("Failed to create lock file " + hbckLockFilePath.getName()
               + ", try=" + (retryCounter.getAttemptTimes() + 1) + " of "
@@ -513,7 +513,7 @@ public class HBaseFsck extends Configured implements Closeable {
       do {
         try {
           IOUtils.closeQuietly(hbckOutFd);
-          FSUtils.delete(FSUtils.getCurrentFileSystem(getConf()), HBCK_LOCK_PATH, true);
+          CommonFSUtils.delete(CommonFSUtils.getCurrentFileSystem(getConf()), HBCK_LOCK_PATH, true);
           LOG.info("Finishing hbck");
           return;
         } catch (IOException ioe) {
@@ -892,9 +892,9 @@ public class HBaseFsck extends Configured implements Closeable {
       List<RegionInfo> regions = MetaTableAccessor.getAllRegions(connection, true);
       final RegionBoundariesInformation currentRegionBoundariesInformation =
           new RegionBoundariesInformation();
-      Path hbaseRoot = FSUtils.getRootDir(getConf());
+      Path hbaseRoot = CommonFSUtils.getRootDir(getConf());
       for (RegionInfo regionInfo : regions) {
-        Path tableDir = FSUtils.getTableDir(hbaseRoot, regionInfo.getTable());
+        Path tableDir = CommonFSUtils.getTableDir(hbaseRoot, regionInfo.getTable());
         currentRegionBoundariesInformation.regionName = regionInfo.getRegionName();
         // For each region, get the start and stop key from the META and compare them to the
         // same information from the Stores.
@@ -1131,7 +1131,7 @@ public class HBaseFsck extends Configured implements Closeable {
   private void offlineReferenceFileRepair() throws IOException, InterruptedException {
     clearState();
     Configuration conf = getConf();
-    Path hbaseRoot = FSUtils.getRootDir(conf);
+    Path hbaseRoot = CommonFSUtils.getRootDir(conf);
     FileSystem fs = hbaseRoot.getFileSystem(conf);
     LOG.info("Computing mapping of all store files");
     Map<String, Path> allFiles = FSUtils.getTableStoreFilePathMap(fs, hbaseRoot,
@@ -1184,7 +1184,7 @@ public class HBaseFsck extends Configured implements Closeable {
    */
   private void offlineHLinkFileRepair() throws IOException, InterruptedException {
     Configuration conf = getConf();
-    Path hbaseRoot = FSUtils.getRootDir(conf);
+    Path hbaseRoot = CommonFSUtils.getRootDir(conf);
     FileSystem fs = hbaseRoot.getFileSystem(conf);
     LOG.info("Computing mapping of all link files");
     Map<String, Path> allFiles = FSUtils
@@ -1313,7 +1313,7 @@ public class HBaseFsck extends Configured implements Closeable {
       }
     }
 
-    Path hbaseRoot = FSUtils.getRootDir(getConf());
+    Path hbaseRoot = CommonFSUtils.getRootDir(getConf());
     FileSystem fs = hbaseRoot.getFileSystem(getConf());
     // serialized table info gathering.
     for (HbckRegionInfo hbi: hbckRegionInfos) {
@@ -1512,7 +1512,7 @@ public class HBaseFsck extends Configured implements Closeable {
 
   Path getSidelineDir() throws IOException {
     if (sidelineDir == null) {
-      Path hbaseDir = FSUtils.getRootDir(getConf());
+      Path hbaseDir = CommonFSUtils.getRootDir(getConf());
       Path hbckDir = new Path(hbaseDir, HConstants.HBCK_SIDELINEDIR_NAME);
       sidelineDir = new Path(hbckDir, hbaseDir.getName() + "-"
           + startMillis);
@@ -1549,7 +1549,7 @@ public class HBaseFsck extends Configured implements Closeable {
     if (parentDir != null) {
       rootDir = new Path(rootDir, parentDir);
     }
-    Path sidelineTableDir= FSUtils.getTableDir(rootDir, tableName);
+    Path sidelineTableDir= CommonFSUtils.getTableDir(rootDir, tableName);
     Path sidelineRegionDir = new Path(sidelineTableDir, regionDir.getName());
     fs.mkdirs(sidelineRegionDir);
     boolean success = false;
@@ -1635,7 +1635,7 @@ public class HBaseFsck extends Configured implements Closeable {
    * regionInfoMap
    */
   public void loadHdfsRegionDirs() throws IOException, InterruptedException {
-    Path rootDir = FSUtils.getRootDir(getConf());
+    Path rootDir = CommonFSUtils.getRootDir(getConf());
     FileSystem fs = rootDir.getFileSystem(getConf());
 
     // list all tables from HDFS
@@ -1645,7 +1645,7 @@ public class HBaseFsck extends Configured implements Closeable {
 
     List<Path> paths = FSUtils.getTableDirs(fs, rootDir);
     for (Path path : paths) {
-      TableName tableName = FSUtils.getTableName(path);
+      TableName tableName = CommonFSUtils.getTableName(path);
        if ((!checkMetaOnly &&
            isTableIncluded(tableName)) ||
            tableName.equals(TableName.META_TABLE_NAME)) {
@@ -1939,7 +1939,7 @@ public class HBaseFsck extends Configured implements Closeable {
       return;
     }
 
-    Path hbaseDir = FSUtils.getRootDir(getConf());
+    Path hbaseDir = CommonFSUtils.getRootDir(getConf());
     FileSystem fs = hbaseDir.getFileSystem(getConf());
     UserProvider userProvider = UserProvider.instantiate(getConf());
     UserGroupInformation ugi = userProvider.getCurrent().getUGI();
@@ -3604,9 +3604,9 @@ public class HBaseFsck extends Configured implements Closeable {
   public static void main(String[] args) throws Exception {
     // create a fsck object
     Configuration conf = HBaseConfiguration.create();
-    Path hbasedir = FSUtils.getRootDir(conf);
+    Path hbasedir = CommonFSUtils.getRootDir(conf);
     URI defaultFs = hbasedir.getFileSystem(conf).getUri();
-    FSUtils.setFsDefault(conf, new Path(defaultFs));
+    CommonFSUtils.setFsDefault(conf, new Path(defaultFs));
     int ret = ToolRunner.run(new HBaseFsckTool(conf), args);
     System.exit(ret);
   }
@@ -3811,13 +3811,13 @@ public class HBaseFsck extends Configured implements Closeable {
         setHFileCorruptionChecker(hfcc); // so we can get result
         Collection<TableName> tables = getIncludedTables();
         Collection<Path> tableDirs = new ArrayList<>();
-        Path rootdir = FSUtils.getRootDir(getConf());
+        Path rootdir = CommonFSUtils.getRootDir(getConf());
         if (tables.size() > 0) {
           for (TableName t : tables) {
-            tableDirs.add(FSUtils.getTableDir(rootdir, t));
+            tableDirs.add(CommonFSUtils.getTableDir(rootdir, t));
           }
         } else {
-          tableDirs = FSUtils.getTableDirs(FSUtils.getCurrentFileSystem(getConf()), rootdir);
+          tableDirs = FSUtils.getTableDirs(CommonFSUtils.getCurrentFileSystem(getConf()), rootdir);
         }
         hfcc.checkTables(tableDirs);
         hfcc.report(errors);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java
index dd40954..d60f2cb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java
@@ -186,7 +186,7 @@ public class HBaseFsckRepair {
   public static HRegion createHDFSRegionDir(Configuration conf,
       RegionInfo hri, TableDescriptor htd) throws IOException {
     // Create HRegion
-    Path root = FSUtils.getRootDir(conf);
+    Path root = CommonFSUtils.getRootDir(conf);
     HRegion region = HRegion.createHRegion(hri, root, conf, htd, null);
 
     // Close the new region to flush to disk. Close log file too.
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HFileArchiveUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HFileArchiveUtil.java
index 76d46f6..684d90d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HFileArchiveUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HFileArchiveUtil.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.util;
 
 import java.io.IOException;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HConstants;
@@ -77,10 +76,9 @@ public final class HFileArchiveUtil {
    * @return {@link Path} to the directory to archive the given store or <tt>null</tt> if it should
    *         not be archived
    */
-  public static Path getStoreArchivePath(Configuration conf,
-                                         RegionInfo region,
-      byte[] family) throws IOException {
-    Path rootDir = FSUtils.getRootDir(conf);
+  public static Path getStoreArchivePath(Configuration conf, RegionInfo region, byte[] family)
+    throws IOException {
+    Path rootDir = CommonFSUtils.getRootDir(conf);
     Path tableArchiveDir = getTableArchivePath(rootDir, region.getTable());
     return HStore.getStoreHomedir(tableArchiveDir, region, family);
   }
@@ -147,7 +145,7 @@ public final class HFileArchiveUtil {
    * @return {@link Path} to the archive directory for the table
    */
   public static Path getTableArchivePath(final Path rootdir, final TableName tableName) {
-    return FSUtils.getTableDir(getArchivePath(rootdir), tableName);
+    return CommonFSUtils.getTableDir(getArchivePath(rootdir), tableName);
   }
 
   /**
@@ -158,10 +156,9 @@ public final class HFileArchiveUtil {
    * @param tableName Name of the table to be archived. Cannot be null.
    * @return {@link Path} to the archive directory for the table
    */
-  public static Path getTableArchivePath(final Configuration conf,
-                                         final TableName tableName)
-      throws IOException {
-    return FSUtils.getTableDir(getArchivePath(conf), tableName);
+  public static Path getTableArchivePath(final Configuration conf, final TableName tableName)
+    throws IOException {
+    return CommonFSUtils.getTableDir(getArchivePath(conf), tableName);
   }
 
   /**
@@ -173,7 +170,7 @@ public final class HFileArchiveUtil {
    * @throws IOException if an unexpected error occurs
    */
   public static Path getArchivePath(Configuration conf) throws IOException {
-    return getArchivePath(FSUtils.getRootDir(conf));
+    return getArchivePath(CommonFSUtils.getRootDir(conf));
   }
 
   /**
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HbckRegionInfo.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HbckRegionInfo.java
index de1d179..acd207a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HbckRegionInfo.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HbckRegionInfo.java
@@ -21,7 +21,6 @@ import java.io.IOException;
 import java.util.Arrays;
 import java.util.Comparator;
 import java.util.List;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -167,7 +166,7 @@ public class HbckRegionInfo implements KeyRange {
       // we are only guaranteed to have a path and not an HRI for hdfsEntry,
       // so we get the name from the Path
       Path tableDir = this.hdfsEntry.regionDir.getParent();
-      return FSUtils.getTableName(tableDir);
+      return CommonFSUtils.getTableName(tableDir);
     } else {
       // return the info from the first online/deployed hri
       for (OnlineEntry e : deployedEntries) {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java
index 657243a..da84c15 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java
@@ -457,8 +457,8 @@ public class RegionSplitter {
       // Max outstanding splits. default == 50% of servers
       final int MAX_OUTSTANDING = Math.max(getRegionServerCount(connection) / 2, minOS);
 
-      Path hbDir = FSUtils.getRootDir(conf);
-      Path tableDir = FSUtils.getTableDir(hbDir, tableName);
+      Path hbDir = CommonFSUtils.getRootDir(conf);
+      Path tableDir = CommonFSUtils.getTableDir(hbDir, tableName);
       Path splitFile = new Path(tableDir, "_balancedSplit");
       FileSystem fs = FileSystem.get(conf);
 
@@ -783,10 +783,9 @@ public class RegionSplitter {
    * @throws IOException if a remote or network exception occurs
    */
   private static Pair<Path, Path> getTableDirAndSplitFile(final Configuration conf,
-      final TableName tableName)
-  throws IOException {
-    Path hbDir = FSUtils.getRootDir(conf);
-    Path tableDir = FSUtils.getTableDir(hbDir, tableName);
+    final TableName tableName) throws IOException {
+    Path hbDir = CommonFSUtils.getRootDir(conf);
+    Path tableDir = CommonFSUtils.getTableDir(hbDir, tableName);
     Path splitFile = new Path(tableDir, "_balancedSplit");
     return new Pair<>(tableDir, splitFile);
   }
@@ -846,8 +845,7 @@ public class RegionSplitter {
       fs.rename(tmpFile, splitFile);
     } else {
       LOG.debug("_balancedSplit file found. Replay log to restore state...");
-      FSUtils.getInstance(fs, connection.getConfiguration())
-        .recoverFileLease(fs, splitFile, connection.getConfiguration(), null);
+      FSUtils.recoverFileLease(fs, splitFile, connection.getConfiguration(), null);
 
       // parse split file and process remaining splits
       FSDataInputStream tmpIn = fs.open(splitFile);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/compaction/MajorCompactionRequest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/compaction/MajorCompactionRequest.java
index cf5fcd9..aecfc37 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/compaction/MajorCompactionRequest.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/compaction/MajorCompactionRequest.java
@@ -32,10 +32,12 @@ import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.yetus.audience.InterfaceAudience;
 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.Sets;
 
@@ -174,9 +176,9 @@ class MajorCompactionRequest {
   HRegionFileSystem getFileSystem(Connection connection) throws IOException {
     Admin admin = connection.getAdmin();
     return HRegionFileSystem.openRegionFromFileSystem(admin.getConfiguration(),
-        FSUtils.getCurrentFileSystem(admin.getConfiguration()),
-        FSUtils.getTableDir(FSUtils.getRootDir(admin.getConfiguration()), region.getTable()),
-        region, true);
+      CommonFSUtils.getCurrentFileSystem(admin.getConfiguration()), CommonFSUtils.getTableDir(
+        CommonFSUtils.getRootDir(admin.getConfiguration()), region.getTable()),
+      region, true);
   }
 
   @Override
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/HFileCorruptionChecker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/HFileCorruptionChecker.java
index f64f991..9a398f2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/HFileCorruptionChecker.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/HFileCorruptionChecker.java
@@ -30,11 +30,6 @@ import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
 import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.hadoop.hbase.util.HbckErrorReporter;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -45,10 +40,15 @@ import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.CorruptHFileException;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.mob.MobUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.FSUtils.FamilyDirFilter;
 import org.apache.hadoop.hbase.util.FSUtils.HFileFilter;
 import org.apache.hadoop.hbase.util.FSUtils.RegionDirFilter;
+import org.apache.hadoop.hbase.util.HbckErrorReporter;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * This class marches through all of the region's hfiles and verifies that
@@ -142,7 +142,7 @@ public class HFileCorruptionChecker {
     Path tableDir = regionDir.getParent();
 
     // build up the corrupted dirs structure
-    Path corruptBaseDir = new Path(FSUtils.getRootDir(conf), HConstants.CORRUPT_DIR_NAME);
+    Path corruptBaseDir = new Path(CommonFSUtils.getRootDir(conf), HConstants.CORRUPT_DIR_NAME);
     if (conf.get("hbase.hfile.quarantine.dir") != null) {
       LOG.warn("hbase.hfile.quarantine.dir is deprecated. Default to " + corruptBaseDir);
     }
@@ -433,7 +433,7 @@ public class HFileCorruptionChecker {
    * @return An instance of MobRegionDirChecker.
    */
   private MobRegionDirChecker createMobRegionDirChecker(Path tableDir) {
-    TableName tableName = FSUtils.getTableName(tableDir);
+    TableName tableName = CommonFSUtils.getTableName(tableDir);
     Path mobDir = MobUtils.getMobRegionPath(conf, tableName);
     return new MobRegionDirChecker(mobDir);
   }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java
index 3f907db..c85a688 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL;
 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
 import org.apache.hadoop.hbase.util.CancelableProgressable;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.LeaseNotRecoveredException;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -433,8 +434,8 @@ public abstract class AbstractFSWALProvider<T extends AbstractFSWAL<?>> implemen
    * @throws IOException exception
    */
   public static Path getArchivedLogPath(Path path, Configuration conf) throws IOException {
-    Path walRootDir = FSUtils.getWALRootDir(conf);
-    Path oldLogDir = new Path(walRootDir, HConstants.HREGION_OLDLOGDIR_NAME);
+    Path rootDir = CommonFSUtils.getWALRootDir(conf);
+    Path oldLogDir = new Path(rootDir, HConstants.HREGION_OLDLOGDIR_NAME);
     if (conf.getBoolean(SEPARATE_OLDLOGDIR, DEFAULT_SEPARATE_OLDLOGDIR)) {
       ServerName serverName = getServerNameFromWALDirectoryName(path);
       if (serverName == null) {
@@ -444,7 +445,7 @@ public abstract class AbstractFSWALProvider<T extends AbstractFSWAL<?>> implemen
       oldLogDir = new Path(oldLogDir, serverName.getServerName());
     }
     Path archivedLogLocation = new Path(oldLogDir, path.getName());
-    final FileSystem fs = FSUtils.getWALFileSystem(conf);
+    final FileSystem fs = CommonFSUtils.getWALFileSystem(conf);
 
     if (fs.exists(archivedLogLocation)) {
       LOG.info("Log " + path + " was moved to " + archivedLogLocation);
@@ -513,9 +514,8 @@ public abstract class AbstractFSWALProvider<T extends AbstractFSWAL<?>> implemen
   // For HBASE-15019
   private static void recoverLease(final Configuration conf, final Path path) {
     try {
-      final FileSystem dfs = FSUtils.getCurrentFileSystem(conf);
-      FSUtils fsUtils = FSUtils.getInstance(dfs, conf);
-      fsUtils.recoverFileLease(dfs, path, conf, new CancelableProgressable() {
+      final FileSystem dfs = CommonFSUtils.getCurrentFileSystem(conf);
+      FSUtils.recoverFileLease(dfs, path, conf, new CancelableProgressable() {
         @Override
         public boolean progress() {
           LOG.debug("Still trying to recover WAL lease: " + path);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java
index a60a4ce..98773c2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java
@@ -25,7 +25,6 @@ import java.util.OptionalLong;
 import java.util.Set;
 import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.atomic.AtomicBoolean;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
@@ -35,7 +34,7 @@ import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl.WriteEntry;
 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
 import org.apache.hadoop.hbase.regionserver.wal.WALCoprocessorHost;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -63,7 +62,7 @@ class DisabledWALProvider implements WALProvider {
     if (null == providerId) {
       providerId = "defaultDisabled";
     }
-    disabled = new DisabledWAL(new Path(FSUtils.getWALRootDir(conf), providerId), conf, null);
+    disabled = new DisabledWAL(new Path(CommonFSUtils.getWALRootDir(conf), providerId), conf, null);
   }
 
   @Override
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALPrettyPrinter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALPrettyPrinter.java
index 8c90a4b..6084e0a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALPrettyPrinter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALPrettyPrinter.java
@@ -38,12 +38,13 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.GsonUtil;
 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.gson.Gson;
 import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
 import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLineParser;
@@ -479,7 +480,7 @@ public class WALPrettyPrinter {
     }
     // get configuration, file system, and process the given files
     Configuration conf = HBaseConfiguration.create();
-    FSUtils.setFsDefault(conf, FSUtils.getRootDir(conf));
+    CommonFSUtils.setFsDefault(conf, CommonFSUtils.getRootDir(conf));
 
     // begin output
     printer.beginPersistentOutput();
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitUtil.java
index a262aa7..2b38494 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitUtil.java
@@ -28,7 +28,6 @@ import java.util.TreeSet;
 import java.util.UUID;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
-
 import org.apache.commons.lang3.ArrayUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
@@ -49,6 +48,7 @@ import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.ConcurrentMapUtils.IOExceptionSupplier;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.Pair;
@@ -91,10 +91,10 @@ public final class WALSplitUtil {
    * @throws IOException
    */
   public static void finishSplitLogFile(String logfile, Configuration conf) throws IOException {
-    Path walDir = FSUtils.getWALRootDir(conf);
+    Path walDir = CommonFSUtils.getWALRootDir(conf);
     Path oldLogDir = new Path(walDir, HConstants.HREGION_OLDLOGDIR_NAME);
     Path walPath;
-    if (FSUtils.isStartingWithPath(walDir, logfile)) {
+    if (CommonFSUtils.isStartingWithPath(walDir, logfile)) {
       walPath = new Path(logfile);
     } else {
       walPath = new Path(walDir, logfile);
@@ -123,7 +123,8 @@ public final class WALSplitUtil {
    */
   private static void archiveWALs(final List<Path> corruptedWALs, final List<Path> processedWALs,
       final Path oldWALDir, final FileSystem walFS, final Configuration conf) throws IOException {
-    final Path corruptDir = new Path(FSUtils.getWALRootDir(conf), HConstants.CORRUPT_DIR_NAME);
+    final Path corruptDir =
+      new Path(CommonFSUtils.getWALRootDir(conf), HConstants.CORRUPT_DIR_NAME);
     if (conf.get("hbase.regionserver.hlog.splitlog.corrupt.dir") != null) {
       LOG.warn("hbase.regionserver.hlog.splitlog.corrupt.dir is deprecated. Default to {}",
         corruptDir);
@@ -149,7 +150,7 @@ public final class WALSplitUtil {
     for (Path p : processedWALs) {
       Path newPath = AbstractFSWAL.getWALArchivePath(oldWALDir, p);
       if (walFS.exists(p)) {
-        if (!FSUtils.renameAndSetModifyTime(walFS, p, newPath)) {
+        if (!CommonFSUtils.renameAndSetModifyTime(walFS, p, newPath)) {
           LOG.warn("Unable to move {} to {}", p, newPath);
         } else {
           LOG.info("Archived processed log {} to {}", p, newPath);
@@ -176,8 +177,8 @@ public final class WALSplitUtil {
   @VisibleForTesting
   static Path getRegionSplitEditsPath(TableName tableName, byte[] encodedRegionName, long seqId,
       String fileNameBeingSplit, String tmpDirName, Configuration conf) throws IOException {
-    FileSystem walFS = FSUtils.getWALFileSystem(conf);
-    Path tableDir = FSUtils.getWALTableDir(conf, tableName);
+    FileSystem walFS = CommonFSUtils.getWALFileSystem(conf);
+    Path tableDir = CommonFSUtils.getWALTableDir(conf, tableName);
     String encodedRegionNameStr = Bytes.toString(encodedRegionName);
     Path regionDir = HRegion.getRegionDir(tableDir, encodedRegionNameStr);
     Path dir = getRegionDirRecoveredEditsDir(regionDir);
@@ -251,12 +252,12 @@ public final class WALSplitUtil {
     // Only default replica region can reach here, so we can use regioninfo
     // directly without converting it to default replica's regioninfo.
     Path regionWALDir =
-        FSUtils.getWALRegionDir(conf, regionInfo.getTable(), regionInfo.getEncodedName());
-    Path regionDir = FSUtils.getRegionDirFromRootDir(FSUtils.getRootDir(conf), regionInfo);
+      CommonFSUtils.getWALRegionDir(conf, regionInfo.getTable(), regionInfo.getEncodedName());
+    Path regionDir = FSUtils.getRegionDirFromRootDir(CommonFSUtils.getRootDir(conf), regionInfo);
     Path wrongRegionWALDir =
-        FSUtils.getWrongWALRegionDir(conf, regionInfo.getTable(), regionInfo.getEncodedName());
-    FileSystem walFs = FSUtils.getWALFileSystem(conf);
-    FileSystem rootFs = FSUtils.getRootDirFileSystem(conf);
+      CommonFSUtils.getWrongWALRegionDir(conf, regionInfo.getTable(), regionInfo.getEncodedName());
+    FileSystem walFs = CommonFSUtils.getWALFileSystem(conf);
+    FileSystem rootFs = CommonFSUtils.getRootDirFileSystem(conf);
     NavigableSet<Path> files = getSplitEditFilesSorted(walFs, regionWALDir);
     if (!files.isEmpty()) {
       return true;
@@ -280,16 +281,17 @@ public final class WALSplitUtil {
    */
   @Deprecated
   public static long getMaxRegionSequenceId(Configuration conf, RegionInfo region,
-      IOExceptionSupplier<FileSystem> rootFsSupplier, IOExceptionSupplier<FileSystem> walFsSupplier)
-      throws IOException {
+    IOExceptionSupplier<FileSystem> rootFsSupplier, IOExceptionSupplier<FileSystem> walFsSupplier)
+    throws IOException {
     FileSystem rootFs = rootFsSupplier.get();
     FileSystem walFs = walFsSupplier.get();
-    Path regionWALDir = FSUtils.getWALRegionDir(conf, region.getTable(), region.getEncodedName());
+    Path regionWALDir =
+      CommonFSUtils.getWALRegionDir(conf, region.getTable(), region.getEncodedName());
     // This is the old place where we store max sequence id file
-    Path regionDir = FSUtils.getRegionDirFromRootDir(FSUtils.getRootDir(conf), region);
+    Path regionDir = FSUtils.getRegionDirFromRootDir(CommonFSUtils.getRootDir(conf), region);
     // This is for HBASE-20734, where we use a wrong directory, see HBASE-22617 for more details.
     Path wrongRegionWALDir =
-      FSUtils.getWrongWALRegionDir(conf, region.getTable(), region.getEncodedName());
+      CommonFSUtils.getWrongWALRegionDir(conf, region.getTable(), region.getEncodedName());
     long maxSeqId = getMaxRegionSequenceId(walFs, regionWALDir);
     maxSeqId = Math.max(maxSeqId, getMaxRegionSequenceId(rootFs, regionDir));
     maxSeqId = Math.max(maxSeqId, getMaxRegionSequenceId(walFs, wrongRegionWALDir));
@@ -310,7 +312,7 @@ public final class WALSplitUtil {
     if (!walFS.exists(editsdir)) {
       return filesSorted;
     }
-    FileStatus[] files = FSUtils.listStatus(walFS, editsdir, new PathFilter() {
+    FileStatus[] files = CommonFSUtils.listStatus(walFS, editsdir, new PathFilter() {
       @Override
       public boolean accept(Path p) {
         boolean result = false;
@@ -579,8 +581,8 @@ public final class WALSplitUtil {
    */
   static Path tryCreateRecoveredHFilesDir(FileSystem rootFS, Configuration conf,
       TableName tableName, String encodedRegionName, String familyName) throws IOException {
-    Path rootDir = FSUtils.getRootDir(conf);
-    Path regionDir = FSUtils.getRegionDirFromTableDir(FSUtils.getTableDir(rootDir, tableName),
+    Path rootDir = CommonFSUtils.getRootDir(conf);
+    Path regionDir = FSUtils.getRegionDirFromTableDir(CommonFSUtils.getTableDir(rootDir, tableName),
       encodedRegionName);
     Path dir = getRecoveredHFilesDir(regionDir, familyName);
     if (!rootFS.exists(dir) && !rootFS.mkdirs(dir)) {
@@ -602,6 +604,6 @@ public final class WALSplitUtil {
   public static FileStatus[] getRecoveredHFiles(final FileSystem rootFS,
       final Path regionDir, String familyName) throws IOException {
     Path dir = getRecoveredHFilesDir(regionDir, familyName);
-    return FSUtils.listStatus(rootFS, dir);
+    return CommonFSUtils.listStatus(rootFS, dir);
   }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
index 0e192f7..a56cdb8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
@@ -51,6 +51,7 @@ import org.apache.hadoop.hbase.regionserver.RegionServerServices;
 import org.apache.hadoop.hbase.regionserver.wal.WALCellCodec;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.CancelableProgressable;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSTableDescriptors;
 import org.apache.hadoop.hbase.util.FSUtils;
@@ -193,7 +194,7 @@ public class WALSplitter {
       Configuration conf, CancelableProgressable reporter, LastSequenceId idChecker,
       SplitLogWorkerCoordination splitLogWorkerCoordination, WALFactory factory,
       RegionServerServices rsServices) throws IOException {
-    Path rootDir = FSUtils.getRootDir(conf);
+    Path rootDir = CommonFSUtils.getRootDir(conf);
     FileSystem rootFS = rootDir.getFileSystem(conf);
     WALSplitter s = new WALSplitter(factory, conf, walDir, walFS, rootDir, rootFS, idChecker,
         splitLogWorkerCoordination, rsServices);
@@ -207,7 +208,7 @@ public class WALSplitter {
   @VisibleForTesting
   public static List<Path> split(Path walDir, Path logDir, Path oldLogDir, FileSystem walFS,
       Configuration conf, final WALFactory factory) throws IOException {
-    Path rootDir = FSUtils.getRootDir(conf);
+    Path rootDir = CommonFSUtils.getRootDir(conf);
     FileSystem rootFS = rootDir.getFileSystem(conf);
     final FileStatus[] logfiles =
         SplitLogManager.getFileList(conf, Collections.singletonList(logDir), null);
@@ -392,7 +393,7 @@ public class WALSplitter {
     }
 
     try {
-      FSUtils.getInstance(walFS, conf).recoverFileLease(walFS, path, conf, reporter);
+      FSUtils.recoverFileLease(walFS, path, conf, reporter);
       try {
         in = getReader(path, reporter);
       } catch (EOFException e) {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java
index d3aff18..ea162fc2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java
@@ -21,7 +21,8 @@ package org.apache.hadoop.hbase;
 import java.io.IOException;
 import java.nio.charset.StandardCharsets;
 import java.util.NavigableMap;
-
+import junit.framework.AssertionFailedError;
+import junit.framework.TestCase;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -35,15 +36,12 @@ import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.RegionAsTable;
 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.hdfs.MiniDFSCluster;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import junit.framework.AssertionFailedError;
-import junit.framework.TestCase;
-
 /**
  * Abstract HBase test class.  Initializes a few things that can come in handly
  * like an HBaseConfiguration and filesystem.
@@ -117,7 +115,7 @@ public abstract class HBaseTestCase extends TestCase {
           fs.delete(testDir, true);
         }
       } else {
-        testDir = FSUtils.getRootDir(conf);
+        testDir = CommonFSUtils.getRootDir(conf);
       }
     } catch (Exception e) {
       LOG.error(HBaseMarkers.FATAL, "error during setup", e);
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 4c4e7c4..4c22e18 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
@@ -654,7 +654,7 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
       return;
     }
     FileSystem fs = this.dfsCluster.getFileSystem();
-    FSUtils.setFsDefault(this.conf, new Path(fs.getUri()));
+    CommonFSUtils.setFsDefault(this.conf, new Path(fs.getUri()));
 
     // re-enable this check with dfs
     conf.unset(CommonFSUtils.UNSAFE_STREAM_CAPABILITY_ENFORCE);
@@ -817,7 +817,7 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
       this.dfsCluster.shutdown();
       dfsCluster = null;
       dataTestDirOnTestFS = null;
-      FSUtils.setFsDefault(this.conf, new Path("file:///"));
+      CommonFSUtils.setFsDefault(this.conf, new Path("file:///"));
     }
   }
 
@@ -1439,7 +1439,7 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
   public Path createRootDir(boolean create) throws IOException {
     FileSystem fs = FileSystem.get(this.conf);
     Path hbaseRootdir = getDefaultRootDirPath(create);
-    FSUtils.setRootDir(this.conf, hbaseRootdir);
+    CommonFSUtils.setRootDir(this.conf, hbaseRootdir);
     fs.mkdirs(hbaseRootdir);
     FSUtils.setVersion(fs, hbaseRootdir);
     return hbaseRootdir;
@@ -1467,7 +1467,7 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
   public Path createWALRootDir() throws IOException {
     FileSystem fs = FileSystem.get(this.conf);
     Path walDir = getNewDataTestDirOnTestFS();
-    FSUtils.setWALRootDir(this.conf, walDir);
+    CommonFSUtils.setWALRootDir(this.conf, walDir);
     fs.mkdirs(walDir);
     return walDir;
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHColumnDescriptorDefaultVersions.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHColumnDescriptorDefaultVersions.java
index e7a9b45..4305091 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHColumnDescriptorDefaultVersions.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHColumnDescriptorDefaultVersions.java
@@ -28,8 +28,8 @@ import org.apache.hadoop.hbase.master.MasterFileSystem;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 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.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
@@ -162,7 +162,7 @@ public class TestHColumnDescriptorDefaultVersions {
 
     // Verify descriptor from HDFS
     MasterFileSystem mfs = TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterFileSystem();
-    Path tableDir = FSUtils.getTableDir(mfs.getRootDir(), tableName);
+    Path tableDir = CommonFSUtils.getTableDir(mfs.getRootDir(), tableName);
     TableDescriptor td = FSTableDescriptors.getTableDescriptorFromFs(mfs.getFileSystem(), tableDir);
     hcds = td.getColumnFamilies();
     verifyHColumnDescriptor(expected, hcds, tableName, families);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java
index cb899aa..d413939 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java
@@ -40,7 +40,7 @@ import org.apache.hadoop.hbase.master.balancer.SimpleLoadBalancer;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.zookeeper.KeeperException;
@@ -109,7 +109,8 @@ public class TestZooKeeper {
       // Still need to clean things up
       TEST_UTIL.shutdownMiniHBaseCluster();
     } finally {
-      TEST_UTIL.getTestFileSystem().delete(FSUtils.getRootDir(TEST_UTIL.getConfiguration()), true);
+      TEST_UTIL.getTestFileSystem().delete(CommonFSUtils.getRootDir(TEST_UTIL.getConfiguration()),
+        true);
       ZKUtil.deleteNodeRecursively(TEST_UTIL.getZooKeeperWatcher(), "/hbase");
     }
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestHFileArchiving.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestHFileArchiving.java
index 96a3dc5..bdeac86 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestHFileArchiving.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestHFileArchiving.java
@@ -136,21 +136,21 @@ public class TestHFileArchiving {
   @Test
   public void testArchiveStoreFilesDifferentFileSystemsWallWithSchemaPlainRoot() throws Exception {
     String walDir = "mockFS://mockFSAuthority:9876/mockDir/wals/";
-    String baseDir = FSUtils.getRootDir(UTIL.getConfiguration()).toString() + "/";
+    String baseDir = CommonFSUtils.getRootDir(UTIL.getConfiguration()).toString() + "/";
     testArchiveStoreFilesDifferentFileSystems(walDir, baseDir,
       HFileArchiver::archiveStoreFiles);
   }
 
   @Test
   public void testArchiveStoreFilesDifferentFileSystemsWallNullPlainRoot() throws Exception {
-    String baseDir = FSUtils.getRootDir(UTIL.getConfiguration()).toString() + "/";
+    String baseDir = CommonFSUtils.getRootDir(UTIL.getConfiguration()).toString() + "/";
     testArchiveStoreFilesDifferentFileSystems(null, baseDir,
       HFileArchiver::archiveStoreFiles);
   }
 
   @Test
   public void testArchiveStoreFilesDifferentFileSystemsWallAndRootSame() throws Exception {
-    String baseDir = FSUtils.getRootDir(UTIL.getConfiguration()).toString() + "/";
+    String baseDir = CommonFSUtils.getRootDir(UTIL.getConfiguration()).toString() + "/";
     testArchiveStoreFilesDifferentFileSystems("/hbase/wals/", baseDir,
       HFileArchiver::archiveStoreFiles);
   }
@@ -217,7 +217,7 @@ public class TestHFileArchiving {
 
   @Test(expected = IOException.class)
   public void testArchiveRecoveredEditsWrongFS() throws Exception {
-    String baseDir = FSUtils.getRootDir(UTIL.getConfiguration()).toString() + "/";
+    String baseDir = CommonFSUtils.getRootDir(UTIL.getConfiguration()).toString() + "/";
     //Internally, testArchiveStoreFilesDifferentFileSystems will pass a "mockedFS"
     // to HFileArchiver.archiveRecoveredEdits, but since wal-dir is supposedly on same FS
     // as root dir it would lead to conflicting FSes and an IOException is expected.
@@ -306,9 +306,9 @@ public class TestHFileArchiving {
     FileSystem fs = region.getRegionFileSystem().getFileSystem();
 
     // make sure there are some files in the regiondir
-    Path rootDir = FSUtils.getRootDir(fs.getConf());
+    Path rootDir = CommonFSUtils.getRootDir(fs.getConf());
     Path regionDir = FSUtils.getRegionDirFromRootDir(rootDir, region.getRegionInfo());
-    FileStatus[] regionFiles = FSUtils.listStatus(fs, regionDir, null);
+    FileStatus[] regionFiles = CommonFSUtils.listStatus(fs, regionDir, null);
     Assert.assertNotNull("No files in the region directory", regionFiles);
     if (LOG.isDebugEnabled()) {
       List<Path> files = new ArrayList<>();
@@ -325,7 +325,7 @@ public class TestHFileArchiving {
         return dirFilter.accept(file) && !file.getName().toString().startsWith(".");
       }
     };
-    FileStatus[] storeDirs = FSUtils.listStatus(fs, regionDir, nonHidden);
+    FileStatus[] storeDirs = CommonFSUtils.listStatus(fs, regionDir, nonHidden);
     for (FileStatus store : storeDirs) {
       LOG.debug("Deleting store for test");
       fs.delete(store.getPath(), true);
@@ -371,8 +371,8 @@ public class TestHFileArchiving {
     FileSystem fs = UTIL.getTestFileSystem();
 
     // now attempt to depose the regions
-    Path rootDir = FSUtils.getRootDir(UTIL.getConfiguration());
-    Path tableDir = FSUtils.getTableDir(rootDir, regions.get(0).getRegionInfo().getTable());
+    Path rootDir = CommonFSUtils.getRootDir(UTIL.getConfiguration());
+    Path tableDir = CommonFSUtils.getTableDir(rootDir, regions.get(0).getRegionInfo().getTable());
     List<Path> regionDirList = regions.stream()
       .map(region -> FSUtils.getRegionDirFromTableDir(tableDir, region.getRegionInfo()))
       .collect(Collectors.toList());
@@ -409,8 +409,8 @@ public class TestHFileArchiving {
     List<HRegion> regions = initTableForArchivingRegions(tableName);
 
     // now attempt to depose the regions
-    Path rootDir = FSUtils.getRootDir(UTIL.getConfiguration());
-    Path tableDir = FSUtils.getTableDir(rootDir, regions.get(0).getRegionInfo().getTable());
+    Path rootDir = CommonFSUtils.getRootDir(UTIL.getConfiguration());
+    Path tableDir = CommonFSUtils.getTableDir(rootDir, regions.get(0).getRegionInfo().getTable());
     List<Path> regionDirList = regions.stream()
       .map(region -> FSUtils.getRegionDirFromTableDir(tableDir, region.getRegionInfo()))
       .collect(Collectors.toList());
@@ -571,7 +571,7 @@ public class TestHFileArchiving {
     FileSystem fs = UTIL.getTestFileSystem();
 
     Path archiveDir = new Path(rootDir, HConstants.HFILE_ARCHIVE_DIRECTORY);
-    Path regionDir = new Path(FSUtils.getTableDir(new Path("./"),
+    Path regionDir = new Path(CommonFSUtils.getTableDir(new Path("./"),
         TableName.valueOf(name.getMethodName())), "abcdef");
     Path familyDir = new Path(regionDir, "cf");
 
@@ -634,7 +634,7 @@ public class TestHFileArchiving {
 
   @Test
   public void testArchiveRegionWithTableDirNull() throws IOException {
-    Path regionDir = new Path(FSUtils.getTableDir(new Path("./"),
+    Path regionDir = new Path(CommonFSUtils.getTableDir(new Path("./"),
             TableName.valueOf(name.getMethodName())), "xyzabc");
     Path familyDir = new Path(regionDir, "rd");
     Path rootDir = UTIL.getDataTestDirOnTestFS("testCleaningRace");
@@ -651,7 +651,7 @@ public class TestHFileArchiving {
 
   @Test
   public void testArchiveRegionWithRegionDirNull() throws IOException {
-    Path regionDir = new Path(FSUtils.getTableDir(new Path("./"),
+    Path regionDir = new Path(CommonFSUtils.getTableDir(new Path("./"),
             TableName.valueOf(name.getMethodName())), "elgn4nf");
     Path familyDir = new Path(regionDir, "rdar");
     Path rootDir = UTIL.getDataTestDirOnTestFS("testCleaningRace");
@@ -681,7 +681,7 @@ public class TestHFileArchiving {
    * @throws IOException
    */
   private List<String> getAllFileNames(final FileSystem fs, Path archiveDir) throws IOException {
-    FileStatus[] files = FSUtils.listStatus(fs, archiveDir, new PathFilter() {
+    FileStatus[] files = CommonFSUtils.listStatus(fs, archiveDir, new PathFilter() {
       @Override
       public boolean accept(Path p) {
         if (p.getName().contains(HConstants.RECOVERED_EDITS_DIR)) {
@@ -695,13 +695,15 @@ public class TestHFileArchiving {
 
   /** Recursively lookup all the file names under the file[] array **/
   private List<String> recurseOnFiles(FileSystem fs, FileStatus[] files, List<String> fileNames)
-      throws IOException {
+    throws IOException {
     if (files == null || files.length == 0) return fileNames;
 
     for (FileStatus file : files) {
       if (file.isDirectory()) {
-        recurseOnFiles(fs, FSUtils.listStatus(fs, file.getPath(), null), fileNames);
-      } else fileNames.add(file.getPath().getName());
+        recurseOnFiles(fs, CommonFSUtils.listStatus(fs, file.getPath(), null), fileNames);
+      } else {
+        fileNames.add(file.getPath().getName());
+      }
     }
     return fileNames;
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java
index 473e233..bfc1701 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java
@@ -21,6 +21,7 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.mock;
+
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
@@ -49,7 +50,7 @@ import org.apache.hadoop.hbase.regionserver.RegionServerServices;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.HFileArchiveUtil;
 import org.apache.hadoop.hbase.util.StoppableImplementation;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
@@ -119,7 +120,7 @@ public class TestZooKeeperTableArchiveClient {
       // cleanup each of the files/directories registered
       for (Path file : toCleanup) {
       // remove the table and archive directories
-        FSUtils.delete(fs, file, true);
+        CommonFSUtils.delete(fs, file, true);
       }
     } catch (IOException e) {
       LOG.warn("Failure to delete archive directory", e);
@@ -196,7 +197,7 @@ public class TestZooKeeperTableArchiveClient {
     // get the current hfiles in the archive directory
     List<Path> files = getAllFiles(fs, archiveDir);
     if (files == null) {
-      FSUtils.logFileSystemState(fs, UTIL.getDataTestDir(), LOG);
+      CommonFSUtils.logFileSystemState(fs, UTIL.getDataTestDir(), LOG);
       throw new RuntimeException("Didn't archive any files!");
     }
     CountDownLatch finished = setupCleanerWatching(delegate, cleaners, files.size());
@@ -260,7 +261,7 @@ public class TestZooKeeperTableArchiveClient {
     // Should  be archived
     List<Path> files = getAllFiles(fs, archiveDir);
     if (files == null) {
-      FSUtils.logFileSystemState(fs, archiveDir, LOG);
+      CommonFSUtils.logFileSystemState(fs, archiveDir, LOG);
       throw new RuntimeException("Didn't load archive any files!");
     }
 
@@ -323,7 +324,7 @@ public class TestZooKeeperTableArchiveClient {
 
   private Path getTableDir(String tableName) throws IOException {
     Path testDataDir = UTIL.getDataTestDir();
-    FSUtils.setRootDir(UTIL.getConfiguration(), testDataDir);
+    CommonFSUtils.setRootDir(UTIL.getConfiguration(), testDataDir);
     return new Path(testDataDir, tableName);
   }
 
@@ -399,7 +400,7 @@ public class TestZooKeeperTableArchiveClient {
    * @return all files under the directory
    */
   private List<Path> getAllFiles(FileSystem fs, Path dir) throws IOException {
-    FileStatus[] files = FSUtils.listStatus(fs, dir, null);
+    FileStatus[] files = CommonFSUtils.listStatus(fs, dir, null);
     if (files == null) {
       LOG.warn("No files under:" + dir);
       return null;
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RestoreSnapshotFromClientSchemaChangeTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RestoreSnapshotFromClientSchemaChangeTestBase.java
index 7f88f6e..c60bd10 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RestoreSnapshotFromClientSchemaChangeTestBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RestoreSnapshotFromClientSchemaChangeTestBase.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.master.MasterFileSystem;
 import org.apache.hadoop.hbase.regionserver.NoSuchColumnFamilyException;
 import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.junit.Test;
 
@@ -38,7 +39,7 @@ public class RestoreSnapshotFromClientSchemaChangeTestBase
   private Set<String> getFamiliesFromFS(final TableName tableName) throws IOException {
     MasterFileSystem mfs = TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterFileSystem();
     Set<String> families = new HashSet<>();
-    Path tableDir = FSUtils.getTableDir(mfs.getRootDir(), tableName);
+    Path tableDir = CommonFSUtils.getTableDir(mfs.getRootDir(), tableName);
     for (Path regionDir : FSUtils.getRegionDirs(mfs.getFileSystem(), tableDir)) {
       for (Path familyDir : FSUtils.getFamilyDirs(mfs.getFileSystem(), regionDir)) {
         families.add(familyDir.getName());
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 4d8f360..3b3ffc8 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
@@ -43,7 +43,7 @@ import org.apache.hadoop.hbase.regionserver.HStoreFile;
 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.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.Threads;
 import org.junit.ClassRule;
@@ -489,7 +489,7 @@ public class TestAdmin1 extends TestAdminBase {
         for (HStoreFile sf : store.getStorefiles()) {
           assertTrue(sf.toString().contains(fn));
           assertTrue("Column family " + fn + " should have 3 copies",
-            FSUtils.getDefaultReplication(TEST_UTIL.getTestFileSystem(),
+            CommonFSUtils.getDefaultReplication(TEST_UTIL.getTestFileSystem(),
               sf.getPath()) == (sf.getFileInfo().getFileStatus().getReplication()));
         }
 
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..c7e466a 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
@@ -17,6 +17,13 @@
  */
 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;
+
+import java.util.Optional;
+import java.util.Set;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.TableName;
@@ -24,8 +31,8 @@ import org.apache.hadoop.hbase.master.MasterFileSystem;
 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.CommonFSUtils;
 import org.apache.hadoop.hbase.util.FSTableDescriptors;
-import org.apache.hadoop.hbase.util.FSUtils;
 import org.junit.Assert;
 import org.junit.ClassRule;
 import org.junit.Test;
@@ -33,14 +40,6 @@ import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 
-import java.util.Optional;
-import java.util.Set;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
 /**
  * Class to test asynchronous table admin operations
  * @see TestAsyncTableAdminApi This test and it used to be joined it was taking longer than our
@@ -194,7 +193,7 @@ public class TestAsyncTableAdminApi2 extends TestAsyncAdminBase {
 
     // Verify descriptor from HDFS
     MasterFileSystem mfs = TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterFileSystem();
-    Path tableDir = FSUtils.getTableDir(mfs.getRootDir(), tableName);
+    Path tableDir = CommonFSUtils.getTableDir(mfs.getRootDir(), tableName);
     TableDescriptor td = FSTableDescriptors.getTableDescriptorFromFs(mfs.getFileSystem(), tableDir);
     verifyTableDescriptor(td, tableName, families);
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide5.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide5.java
index f74a288..cff4959 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide5.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide5.java
@@ -82,6 +82,7 @@ import org.apache.hadoop.hbase.regionserver.NoSuchColumnFamilyException;
 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.CommonFSUtils;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.junit.AfterClass;
@@ -2573,7 +2574,7 @@ public class TestFromClientSide5 extends FromClientSideBase {
     t.put(new Put(ROW).addColumn(cf1, QUALIFIER, Bytes.toBytes("val1")));
     t.put(new Put(ROW).addColumn(cf2, QUALIFIER, Bytes.toBytes("val2")));
     admin.flush(tableName);
-    Path tableDir = FSUtils.getTableDir(TEST_UTIL.getDefaultRootDirPath(), tableName);
+    Path tableDir = CommonFSUtils.getTableDir(TEST_UTIL.getDefaultRootDirPath(), tableName);
     List<Path> regionDirs = FSUtils.getRegionDirs(TEST_UTIL.getTestFileSystem(), tableDir);
     assertEquals(1, regionDirs.size());
     List<Path> familyDirs = FSUtils.getFamilyDirs(TEST_UTIL.getTestFileSystem(), regionDirs.get(0));
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 d2a0eae..65704ce 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
@@ -43,7 +43,7 @@ import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
 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.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -213,15 +213,15 @@ public class TestSnapshotFromClient {
     UTIL.loadTable(table, TEST_FAM, false);
 
     LOG.debug("FS state before disable:");
-    FSUtils.logFileSystemState(UTIL.getTestFileSystem(),
-      FSUtils.getRootDir(UTIL.getConfiguration()), LOG);
+    CommonFSUtils.logFileSystemState(UTIL.getTestFileSystem(),
+      CommonFSUtils.getRootDir(UTIL.getConfiguration()), LOG);
     // XXX if this is flakey, might want to consider using the async version and looping as
     // disableTable can succeed and still timeout.
     admin.disableTable(TABLE_NAME);
 
     LOG.debug("FS state before snapshot:");
-    FSUtils.logFileSystemState(UTIL.getTestFileSystem(),
-      FSUtils.getRootDir(UTIL.getConfiguration()), LOG);
+    CommonFSUtils.logFileSystemState(UTIL.getTestFileSystem(),
+      CommonFSUtils.getRootDir(UTIL.getConfiguration()), LOG);
 
     // take a snapshot of the disabled table
     final String SNAPSHOT_NAME = "offlineTableSnapshot";
@@ -239,8 +239,8 @@ public class TestSnapshotFromClient {
     FileSystem fs = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getFileSystem();
     Path rootDir = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getRootDir();
     LOG.debug("FS state after snapshot:");
-    FSUtils.logFileSystemState(UTIL.getTestFileSystem(),
-      FSUtils.getRootDir(UTIL.getConfiguration()), LOG);
+    CommonFSUtils.logFileSystemState(UTIL.getTestFileSystem(),
+      CommonFSUtils.getRootDir(UTIL.getConfiguration()), LOG);
     SnapshotTestingUtils.confirmSnapshotValid(
       ProtobufUtil.createHBaseProtosSnapshotDesc(snapshots.get(0)), TABLE_NAME, TEST_FAM,
       rootDir, admin, fs);
@@ -288,13 +288,13 @@ public class TestSnapshotFromClient {
     SnapshotTestingUtils.assertNoSnapshots(admin);
 
     LOG.debug("FS state before disable:");
-    FSUtils.logFileSystemState(UTIL.getTestFileSystem(),
-      FSUtils.getRootDir(UTIL.getConfiguration()), LOG);
+    CommonFSUtils.logFileSystemState(UTIL.getTestFileSystem(),
+      CommonFSUtils.getRootDir(UTIL.getConfiguration()), LOG);
     admin.disableTable(TABLE_NAME);
 
     LOG.debug("FS state before snapshot:");
-    FSUtils.logFileSystemState(UTIL.getTestFileSystem(),
-      FSUtils.getRootDir(UTIL.getConfiguration()), LOG);
+    CommonFSUtils.logFileSystemState(UTIL.getTestFileSystem(),
+      CommonFSUtils.getRootDir(UTIL.getConfiguration()), LOG);
 
     // take a snapshot of the disabled table
     byte[] snapshot = Bytes.toBytes("testOfflineTableSnapshotWithEmptyRegions");
@@ -309,8 +309,8 @@ public class TestSnapshotFromClient {
     FileSystem fs = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getFileSystem();
     Path rootDir = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getRootDir();
     LOG.debug("FS state after snapshot:");
-    FSUtils.logFileSystemState(UTIL.getTestFileSystem(),
-      FSUtils.getRootDir(UTIL.getConfiguration()), LOG);
+    CommonFSUtils.logFileSystemState(UTIL.getTestFileSystem(),
+      CommonFSUtils.getRootDir(UTIL.getConfiguration()), LOG);
 
     List<byte[]> emptyCfs = Lists.newArrayList(TEST_FAM); // no file in the region
     List<byte[]> nonEmptyCfs = Lists.newArrayList();
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 c590215..f7681d7 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
@@ -47,7 +47,7 @@ import org.apache.hadoop.hbase.snapshot.SnapshotManifestV2;
 import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -61,6 +61,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 
 /**
@@ -298,17 +299,15 @@ public class TestSnapshotTemporaryDirectory {
     UTIL.loadTable(table, TEST_FAM, false);
 
     LOG.debug("FS state before disable:");
-    FSUtils
-        .logFileSystemState(UTIL.getTestFileSystem(), FSUtils.getRootDir(UTIL.getConfiguration()),
-            LOG);
+    CommonFSUtils.logFileSystemState(UTIL.getTestFileSystem(),
+      CommonFSUtils.getRootDir(UTIL.getConfiguration()), LOG);
     // XXX if this is flakey, might want to consider using the async version and looping as
     // disableTable can succeed and still timeout.
     admin.disableTable(TABLE_NAME);
 
     LOG.debug("FS state before snapshot:");
-    FSUtils
-        .logFileSystemState(UTIL.getTestFileSystem(), FSUtils.getRootDir(UTIL.getConfiguration()),
-            LOG);
+    CommonFSUtils.logFileSystemState(UTIL.getTestFileSystem(),
+      CommonFSUtils.getRootDir(UTIL.getConfiguration()), LOG);
 
     // take a snapshot of the disabled table
     final String SNAPSHOT_NAME = "offlineTableSnapshot";
@@ -324,9 +323,8 @@ public class TestSnapshotTemporaryDirectory {
     FileSystem fs = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getFileSystem();
     Path rootDir = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getRootDir();
     LOG.debug("FS state after snapshot:");
-    FSUtils
-        .logFileSystemState(UTIL.getTestFileSystem(), FSUtils.getRootDir(UTIL.getConfiguration()),
-            LOG);
+    CommonFSUtils.logFileSystemState(UTIL.getTestFileSystem(),
+      CommonFSUtils.getRootDir(UTIL.getConfiguration()), LOG);
 
     SnapshotTestingUtils
         .confirmSnapshotValid(ProtobufUtil.createHBaseProtosSnapshotDesc(snapshots.get(0)),
@@ -387,15 +385,13 @@ public class TestSnapshotTemporaryDirectory {
     SnapshotTestingUtils.assertNoSnapshots(admin);
 
     LOG.debug("FS state before disable:");
-    FSUtils
-        .logFileSystemState(UTIL.getTestFileSystem(), FSUtils.getRootDir(UTIL.getConfiguration()),
-            LOG);
+    CommonFSUtils.logFileSystemState(UTIL.getTestFileSystem(),
+      CommonFSUtils.getRootDir(UTIL.getConfiguration()), LOG);
     admin.disableTable(TABLE_NAME);
 
     LOG.debug("FS state before snapshot:");
-    FSUtils
-        .logFileSystemState(UTIL.getTestFileSystem(), FSUtils.getRootDir(UTIL.getConfiguration()),
-            LOG);
+    CommonFSUtils.logFileSystemState(UTIL.getTestFileSystem(),
+      CommonFSUtils.getRootDir(UTIL.getConfiguration()), LOG);
 
     // take a snapshot of the disabled table
     byte[] snapshot = Bytes.toBytes("testOfflineTableSnapshotWithEmptyRegion");
@@ -404,15 +400,14 @@ public class TestSnapshotTemporaryDirectory {
 
     // make sure we have the snapshot
     List<SnapshotDescription> snapshots =
-        SnapshotTestingUtils.assertOneSnapshotThatMatches(admin, snapshot, TABLE_NAME);
+      SnapshotTestingUtils.assertOneSnapshotThatMatches(admin, snapshot, TABLE_NAME);
 
     // make sure its a valid snapshot
     FileSystem fs = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getFileSystem();
     Path rootDir = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getRootDir();
     LOG.debug("FS state after snapshot:");
-    FSUtils
-        .logFileSystemState(UTIL.getTestFileSystem(), FSUtils.getRootDir(UTIL.getConfiguration()),
-            LOG);
+    CommonFSUtils.logFileSystemState(UTIL.getTestFileSystem(),
+      CommonFSUtils.getRootDir(UTIL.getConfiguration()), LOG);
 
     List<byte[]> emptyCfs = Lists.newArrayList(TEST_FAM); // no file in the region
     List<byte[]> nonEmptyCfs = Lists.newArrayList();
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 d67d163..e60a2fc 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
@@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
 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.CommonFSUtils;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.HFileArchiveUtil;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
@@ -125,7 +126,7 @@ public class TestTableSnapshotScanner {
     Table table = util.getConnection().getTable(tableName);
     util.loadTable(table, FAMILIES);
 
-    Path rootDir = FSUtils.getRootDir(util.getConfiguration());
+    Path rootDir = CommonFSUtils.getRootDir(util.getConfiguration());
     FileSystem fs = rootDir.getFileSystem(util.getConfiguration());
 
     SnapshotTestingUtils.createSnapshotAndValidate(admin, tableName,
@@ -161,7 +162,7 @@ public class TestTableSnapshotScanner {
       admin.split(tableName, Bytes.toBytes("eee"));
       blockUntilSplitFinished(UTIL, tableName, 2);
 
-      Path rootDir = FSUtils.getRootDir(UTIL.getConfiguration());
+      Path rootDir = CommonFSUtils.getRootDir(UTIL.getConfiguration());
       FileSystem fs = rootDir.getFileSystem(UTIL.getConfiguration());
 
       SnapshotTestingUtils.createSnapshotAndValidate(admin, tableName,
@@ -216,7 +217,7 @@ public class TestTableSnapshotScanner {
       Scan scan = new Scan(bbb, yyy); // limit the scan
 
       Configuration conf = UTIL.getConfiguration();
-      Path rootDir = FSUtils.getRootDir(conf);
+      Path rootDir = CommonFSUtils.getRootDir(conf);
 
       TableSnapshotScanner scanner0 =
           new TableSnapshotScanner(conf, restoreDir, snapshotName, scan);
@@ -340,7 +341,7 @@ public class TestTableSnapshotScanner {
       // wait flush is finished
       UTIL.waitFor(timeout, () -> {
         try {
-          Path tableDir = FSUtils.getTableDir(rootDir, tableName);
+          Path tableDir = CommonFSUtils.getTableDir(rootDir, tableName);
           for (RegionInfo region : regions) {
             Path regionDir = new Path(tableDir, region.getEncodedName());
             for (Path familyDir : FSUtils.getFamilyDirs(fs, regionDir)) {
@@ -384,7 +385,7 @@ public class TestTableSnapshotScanner {
               }
             }
           }
-          Path tableDir = FSUtils.getTableDir(rootDir, tableName);
+          Path tableDir = CommonFSUtils.getTableDir(rootDir, tableName);
           HRegionFileSystem regionFs = HRegionFileSystem
               .openRegionFromFileSystem(UTIL.getConfiguration(), fs, tableDir, mergedRegion, true);
           return !regionFs.hasReferences(admin.getDescriptor(tableName));
@@ -397,7 +398,7 @@ public class TestTableSnapshotScanner {
       UTIL.getMiniHBaseCluster().getMaster().getCatalogJanitor().choreForTesting();
       UTIL.waitFor(timeout, () -> {
         try {
-          Path tableDir = FSUtils.getTableDir(rootDir, tableName);
+          Path tableDir = CommonFSUtils.getTableDir(rootDir, tableName);
           for (FileStatus fileStatus : fs.listStatus(tableDir)) {
             String name = fileStatus.getPath().getName();
             if (name.equals(region0.getEncodedName()) || name.equals(region1.getEncodedName())) {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java
index 9381ef3..6fbd3b5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java
@@ -55,9 +55,9 @@ import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.EnvironmentEdge;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
 import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.wal.WALEdit;
@@ -126,8 +126,8 @@ public class TestWALObserver {
     Path hbaseWALRootDir = TEST_UTIL.getDFSCluster().getFileSystem()
             .makeQualified(new Path("/hbaseLogRoot"));
     LOG.info("hbase.rootdir=" + hbaseRootDir);
-    FSUtils.setRootDir(conf, hbaseRootDir);
-    FSUtils.setWALRootDir(conf, hbaseWALRootDir);
+    CommonFSUtils.setRootDir(conf, hbaseRootDir);
+    CommonFSUtils.setWALRootDir(conf, hbaseWALRootDir);
   }
 
   @AfterClass
@@ -140,8 +140,8 @@ public class TestWALObserver {
     this.conf = HBaseConfiguration.create(TEST_UTIL.getConfiguration());
     // this.cluster = TEST_UTIL.getDFSCluster();
     this.fs = TEST_UTIL.getDFSCluster().getFileSystem();
-    this.hbaseRootDir = FSUtils.getRootDir(conf);
-    this.hbaseWALRootDir = FSUtils.getWALRootDir(conf);
+    this.hbaseRootDir = CommonFSUtils.getRootDir(conf);
+    this.hbaseWALRootDir = CommonFSUtils.getWALRootDir(conf);
     this.oldLogDir = new Path(this.hbaseWALRootDir,
         HConstants.HREGION_OLDLOGDIR_NAME);
     String serverName = ServerName.valueOf(currentTest.getMethodName(), 16010,
@@ -318,8 +318,7 @@ public class TestWALObserver {
     // createBasic3FamilyHRegionInfo(Bytes.toString(tableName));
     RegionInfo hri = RegionInfoBuilder.newBuilder(tableName).build();
 
-    final Path basedir =
-        FSUtils.getTableDir(this.hbaseRootDir, tableName);
+    final Path basedir = CommonFSUtils.getTableDir(this.hbaseRootDir, tableName);
     deleteDir(basedir);
     fs.mkdirs(new Path(basedir, hri.getEncodedName()));
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorderMultiBlocks.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorderMultiBlocks.java
index df760f6..0a4a7f5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorderMultiBlocks.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorderMultiBlocks.java
@@ -42,7 +42,7 @@ import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
@@ -141,8 +141,8 @@ public class TestBlockReorderMultiBlocks {
 
     // Now we need to find the log file, its locations, and look at it
 
-    String rootDir = new Path(FSUtils.getWALRootDir(conf) + "/" + HConstants.HREGION_LOGDIR_NAME +
-            "/" + targetRs.getServerName().toString()).toUri().getPath();
+    String rootDir = new Path(CommonFSUtils.getWALRootDir(conf) + "/" +
+      HConstants.HREGION_LOGDIR_NAME + "/" + targetRs.getServerName().toString()).toUri().getPath();
 
     DistributedFileSystem mdfs = (DistributedFileSystem)
         hbm.getMaster().getMasterFileSystem().getFileSystem();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestInfoServersACL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestInfoServersACL.java
index 7f07fbd..a1e1b91 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestInfoServersACL.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestInfoServersACL.java
@@ -20,11 +20,13 @@ package org.apache.hadoop.hbase.http;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
+
 import java.io.File;
 import java.lang.management.ManagementFactory;
 import java.net.HttpURLConnection;
 import java.net.URL;
 import java.security.PrivilegedExceptionAction;
+import javax.management.ObjectName;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.Path;
@@ -39,9 +41,8 @@ import org.apache.hadoop.hbase.security.HBaseKerberosUtils;
 import org.apache.hadoop.hbase.security.token.TokenProvider;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.minikdc.MiniKdc;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.http.auth.AuthSchemeProvider;
@@ -71,10 +72,6 @@ import org.junit.rules.TestName;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import javax.management.InstanceNotFoundException;
-import javax.management.MBeanInfo;
-import javax.management.ObjectName;
-
 /**
  * Testing info servers for admin acl.
  */
@@ -125,7 +122,7 @@ public class TestInfoServersACL {
         TokenProvider.class.getName());
     UTIL.startMiniDFSCluster(1);
     Path rootdir = UTIL.getDataTestDirOnTestFS("TestInfoServersACL");
-    FSUtils.setRootDir(conf, rootdir);
+    CommonFSUtils.setRootDir(conf, rootdir);
 
     // The info servers do not run in tests by default.
     // Set them to ephemeral ports so they will start
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHFileLink.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHFileLink.java
index c0448b9..8d6070a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHFileLink.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHFileLink.java
@@ -27,7 +27,7 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.testclassification.IOTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.Pair;
 import org.junit.Assert;
 import org.junit.ClassRule;
@@ -103,7 +103,7 @@ public class TestHFileLink {
         TableName.valueOf("ns", name.getMethodName())};
 
     for(TableName refTable : refTables) {
-      Path refTableDir = FSUtils.getTableDir(archiveDir, refTable);
+      Path refTableDir = CommonFSUtils.getTableDir(archiveDir, refTable);
       Path refRegionDir = HRegion.getRegionDir(refTableDir, encodedRegion);
       Path refDir = new Path(refRegionDir, cf);
       Path refLinkDir = new Path(refDir, linkDir);
@@ -115,15 +115,15 @@ public class TestHFileLink {
               TableName.valueOf(name.getMethodName()+ ":" +name.getMethodName())};
 
       for( TableName tableName : tableNames) {
-        Path tableDir = FSUtils.getTableDir(rootDir, tableName);
+        Path tableDir = CommonFSUtils.getTableDir(rootDir, tableName);
         Path regionDir = HRegion.getRegionDir(tableDir, encodedRegion);
         Path cfDir = new Path(regionDir, cf);
 
-        //Verify back reference creation
-        assertEquals(encodedRegion+"."+
-            tableName.getNameAsString().replace(TableName.NAMESPACE_DELIM, '='),
-            HFileLink.createBackReferenceName(FSUtils.getTableName(tableDir).getNameAsString(),
-                encodedRegion));
+        // Verify back reference creation
+        assertEquals(
+          encodedRegion + "." + tableName.getNameAsString().replace(TableName.NAMESPACE_DELIM, '='),
+          HFileLink.createBackReferenceName(CommonFSUtils.getTableName(tableDir).getNameAsString(),
+            encodedRegion));
 
         //verify parsing back reference
         Pair<TableName, String> parsedRef =
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestLocalAsyncOutput.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestLocalAsyncOutput.java
index 406af17..c49e02d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestLocalAsyncOutput.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/asyncfs/TestLocalAsyncOutput.java
@@ -25,7 +25,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.junit.AfterClass;
 import org.junit.ClassRule;
 import org.junit.Test;
@@ -41,7 +41,7 @@ public class TestLocalAsyncOutput {
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestLocalAsyncOutput.class);
+    HBaseClassTestRule.forClass(TestLocalAsyncOutput.class);
 
   private static EventLoopGroup GROUP = new NioEventLoopGroup();
 
@@ -57,7 +57,7 @@ public class TestLocalAsyncOutput {
 
   @Test
   public void test() throws IOException, InterruptedException, ExecutionException,
-      FSUtils.StreamLacksCapabilityException {
+    CommonFSUtils.StreamLacksCapabilityException {
     Path f = new Path(TEST_UTIL.getDataTestDir(), "test");
     FileSystem fs = FileSystem.getLocal(TEST_UTIL.getConfiguration());
     AsyncFSOutput out = AsyncFSOutputHelper.createOutput(fs, f, false, true,
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFilePrettyPrinter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFilePrettyPrinter.java
index f8c85a2..8fab5a3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFilePrettyPrinter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFilePrettyPrinter.java
@@ -23,7 +23,6 @@ import static org.junit.Assert.assertTrue;
 
 import java.io.ByteArrayOutputStream;
 import java.io.PrintStream;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -34,7 +33,6 @@ import org.apache.hadoop.hbase.testclassification.IOTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.CommonFSUtils;
-import org.apache.hadoop.hbase.util.FSUtils;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.ClassRule;
@@ -94,7 +92,7 @@ public class TestHFilePrettyPrinter {
 
   @Test
   public void testHFilePrettyPrinterRootDir() throws Exception {
-    Path rootPath = FSUtils.getRootDir(conf);
+    Path rootPath = CommonFSUtils.getRootDir(conf);
     String rootString = rootPath + rootPath.SEPARATOR;
     Path fileInRootDir = new Path(rootString + "hfile");
     TestHRegionServerBulkLoad.createHFile(fs, fileInRootDir, cf, fam, value, 1000);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/AbstractTestDLS.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/AbstractTestDLS.java
index 485b84b..2b91ba9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/AbstractTestDLS.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/AbstractTestDLS.java
@@ -30,6 +30,7 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
+
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -72,7 +73,7 @@ import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
 import org.apache.hadoop.hbase.util.Threads;
@@ -92,6 +93,7 @@ import org.junit.Test;
 import org.junit.rules.TestName;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 
 /**
@@ -169,7 +171,8 @@ public abstract class AbstractTestDLS {
   @After
   public void after() throws Exception {
     TEST_UTIL.shutdownMiniHBaseCluster();
-    TEST_UTIL.getTestFileSystem().delete(FSUtils.getRootDir(TEST_UTIL.getConfiguration()), true);
+    TEST_UTIL.getTestFileSystem().delete(CommonFSUtils.getRootDir(TEST_UTIL.getConfiguration()),
+      true);
     ZKUtil.deleteNodeRecursively(TEST_UTIL.getZooKeeperWatcher(), "/hbase");
   }
 
@@ -187,7 +190,7 @@ public abstract class AbstractTestDLS {
 
     List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
 
-    Path rootdir = FSUtils.getRootDir(conf);
+    Path rootdir = CommonFSUtils.getRootDir(conf);
 
     int numRegions = 50;
     try (Table t = installTable(numRegions)) {
@@ -222,7 +225,7 @@ public abstract class AbstractTestDLS {
       for (RegionInfo hri : regions) {
         @SuppressWarnings("deprecation")
         Path editsdir = WALSplitUtil
-            .getRegionDirRecoveredEditsDir(FSUtils.getWALRegionDir(conf,
+            .getRegionDirRecoveredEditsDir(CommonFSUtils.getWALRegionDir(conf,
                 tableName, hri.getEncodedName()));
         LOG.debug("Checking edits dir " + editsdir);
         FileStatus[] files = fs.listStatus(editsdir, new PathFilter() {
@@ -319,7 +322,7 @@ public abstract class AbstractTestDLS {
 
     List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
     HRegionServer hrs = findRSToKill(false);
-    Path rootdir = FSUtils.getRootDir(conf);
+    Path rootdir = CommonFSUtils.getRootDir(conf);
     final Path logDir = new Path(rootdir,
         AbstractFSWALProvider.getWALDirectoryName(hrs.getServerName().toString()));
 
@@ -412,7 +415,8 @@ public abstract class AbstractTestDLS {
     startCluster(1);
     final SplitLogManager slm = master.getMasterWalManager().getSplitLogManager();
     final FileSystem fs = master.getMasterFileSystem().getFileSystem();
-    final Path logDir = new Path(new Path(FSUtils.getWALRootDir(conf), HConstants.HREGION_LOGDIR_NAME),
+    final Path logDir =
+      new Path(new Path(CommonFSUtils.getWALRootDir(conf), HConstants.HREGION_LOGDIR_NAME),
         ServerName.valueOf("x", 1, 1).toString());
     fs.mkdirs(logDir);
     ExecutorService executor = null;
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
index c7a462c..467ec05 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
@@ -57,7 +57,7 @@ import org.apache.hadoop.hbase.regionserver.MemStoreLABImpl;
 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.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.HFileArchiveUtil;
 import org.apache.zookeeper.KeeperException;
 import org.junit.After;
@@ -123,7 +123,7 @@ public class TestCatalogJanitor {
     Result r = createResult(parent, splita, splitb);
     // Add a reference under splitA directory so we don't clear out the parent.
     Path rootdir = this.masterServices.getMasterFileSystem().getRootDir();
-    Path tabledir = FSUtils.getTableDir(rootdir, td.getTableName());
+    Path tabledir = CommonFSUtils.getTableDir(rootdir, td.getTableName());
     Path parentdir = new Path(tabledir, parent.getEncodedName());
     Path storedir = HStore.getStoreHomedir(tabledir, splita, td.getColumnFamilies()[0].getName());
     Reference ref = Reference.createTopReference(Bytes.toBytes("ccc"));
@@ -438,8 +438,8 @@ public class TestCatalogJanitor {
     // have to set the root directory since we use it in HFileDisposer to figure out to get to the
     // archive directory. Otherwise, it just seems to pick the first root directory it can find (so
     // the single test passes, but when the full suite is run, things get borked).
-    FSUtils.setRootDir(fs.getConf(), rootdir);
-    Path tabledir = FSUtils.getTableDir(rootdir, td.getTableName());
+    CommonFSUtils.setRootDir(fs.getConf(), rootdir);
+    Path tabledir = CommonFSUtils.getTableDir(rootdir, td.getTableName());
     Path storedir = HStore.getStoreHomedir(tabledir, parent, td.getColumnFamilies()[0].getName());
     Path storeArchive =
         HFileArchiveUtil.getStoreArchivePath(this.masterServices.getConfiguration(), parent,
@@ -476,7 +476,7 @@ public class TestCatalogJanitor {
     assertArchiveEqualToOriginal(storeFiles, archivedStoreFiles, fs);
 
     // cleanup
-    FSUtils.delete(fs, rootdir, true);
+    CommonFSUtils.delete(fs, rootdir, true);
   }
 
   /**
@@ -513,8 +513,8 @@ public class TestCatalogJanitor {
     // Have to set the root directory since we use it in HFileDisposer to figure out to get to the
     // archive directory. Otherwise, it just seems to pick the first root directory it can find (so
     // the single test passes, but when the full suite is run, things get borked).
-    FSUtils.setRootDir(fs.getConf(), rootdir);
-    Path tabledir = FSUtils.getTableDir(rootdir, parent.getTable());
+    CommonFSUtils.setRootDir(fs.getConf(), rootdir);
+    Path tabledir = CommonFSUtils.getTableDir(rootdir, parent.getTable());
     Path storedir = HStore.getStoreHomedir(tabledir, parent, td.getColumnFamilies()[0].getName());
     System.out.println("Old root:" + rootdir);
     System.out.println("Old table:" + tabledir);
@@ -578,8 +578,8 @@ public class TestCatalogJanitor {
     Path testdir = htu.getDataTestDir(subdir);
     FileSystem fs = FileSystem.get(htu.getConfiguration());
     if (fs.exists(testdir)) assertTrue(fs.delete(testdir, true));
-    FSUtils.setRootDir(htu.getConfiguration(), testdir);
-    return FSUtils.getRootDir(htu.getConfiguration()).toString();
+    CommonFSUtils.setRootDir(htu.getConfiguration(), testdir);
+    return CommonFSUtils.getRootDir(htu.getConfiguration()).toString();
   }
 
   private Path createReferences(final MasterServices services,
@@ -587,7 +587,7 @@ public class TestCatalogJanitor {
       final HRegionInfo daughter, final byte [] midkey, final boolean top)
   throws IOException {
     Path rootdir = services.getMasterFileSystem().getRootDir();
-    Path tabledir = FSUtils.getTableDir(rootdir, parent.getTable());
+    Path tabledir = CommonFSUtils.getTableDir(rootdir, parent.getTable());
     Path storedir = HStore.getStoreHomedir(tabledir, daughter,
       td.getColumnFamilies()[0].getName());
     Reference ref =
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystem.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystem.java
index 47cad32..63d303d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystem.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystem.java
@@ -22,7 +22,6 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.util.List;
-
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
@@ -33,7 +32,7 @@ import org.apache.hadoop.hbase.regionserver.HRegion;
 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.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.HFileArchiveTestingUtil;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -76,8 +75,8 @@ public class TestMasterFileSystem {
   public void testFsUriSetProperly() throws Exception {
     HMaster master = UTIL.getMiniHBaseCluster().getMaster();
     MasterFileSystem fs = master.getMasterFileSystem();
-    Path masterRoot = FSUtils.getRootDir(fs.getConfiguration());
-    Path rootDir = FSUtils.getRootDir(fs.getFileSystem().getConf());
+    Path masterRoot = CommonFSUtils.getRootDir(fs.getConfiguration());
+    Path rootDir = CommonFSUtils.getRootDir(fs.getFileSystem().getConf());
     // make sure the fs and the found root dir have the same scheme
     LOG.debug("from fs uri:" + FileSystem.getDefaultUri(fs.getFileSystem().getConf()));
     LOG.debug("from configuration uri:" + FileSystem.getDefaultUri(fs.getConfiguration()));
@@ -111,9 +110,9 @@ public class TestMasterFileSystem {
     // disable the table so that we can manipulate the files
     UTIL.getAdmin().disableTable(tableName);
 
-    final Path tableDir = FSUtils.getTableDir(masterFileSystem.getRootDir(), tableName);
+    final Path tableDir = CommonFSUtils.getTableDir(masterFileSystem.getRootDir(), tableName);
     final Path tempDir = masterFileSystem.getTempDir();
-    final Path tempTableDir = FSUtils.getTableDir(tempDir, tableName);
+    final Path tempTableDir = CommonFSUtils.getTableDir(tempDir, tableName);
     final FileSystem fs = masterFileSystem.getFileSystem();
 
     // move the table to the temporary directory
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystemWithWALDir.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystemWithWALDir.java
index 62e5b5b..262b782 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystemWithWALDir.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystemWithWALDir.java
@@ -25,7 +25,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.StartMiniClusterOption;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
@@ -59,9 +59,9 @@ public class TestMasterFileSystemWithWALDir {
   public void testFsUriSetProperly() throws Exception {
     HMaster master = UTIL.getMiniHBaseCluster().getMaster();
     MasterFileSystem fs = master.getMasterFileSystem();
-    Path masterRoot = FSUtils.getRootDir(fs.getConfiguration());
-    Path rootDir = FSUtils.getRootDir(fs.getFileSystem().getConf());
+    Path masterRoot = CommonFSUtils.getRootDir(fs.getConfiguration());
+    Path rootDir = CommonFSUtils.getRootDir(fs.getFileSystem().getConf());
     assertEquals(masterRoot, rootDir);
-    assertEquals(FSUtils.getWALRootDir(UTIL.getConfiguration()), fs.getWALRootDir());
+    assertEquals(CommonFSUtils.getWALRootDir(UTIL.getConfiguration()), fs.getWALRootDir());
   }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
index e05140b..ae9007c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.master;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.apache.hadoop.hbase.HConstants.ZOOKEEPER_QUORUM;
+
 import java.io.IOException;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
@@ -41,7 +42,7 @@ import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
@@ -90,7 +91,7 @@ public class TestMasterNoCluster {
   public static void setUpBeforeClass() throws Exception {
     Configuration c = TESTUTIL.getConfiguration();
     // We use local filesystem.  Set it so it writes into the testdir.
-    FSUtils.setRootDir(c, TESTUTIL.getDataTestDir());
+    CommonFSUtils.setRootDir(c, TESTUTIL.getDataTestDir());
     DefaultMetricsSystem.setMiniClusterMode(true);
     // Startup a mini zk cluster.
     TESTUTIL.startMiniZKCluster();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNotCarryTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNotCarryTable.java
index 4eefe49..e62766d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNotCarryTable.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNotCarryTable.java
@@ -18,15 +18,13 @@
 package org.apache.hadoop.hbase.master;
 
 import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNull;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.regionserver.ChunkCreator;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
@@ -52,7 +50,7 @@ public class TestMasterNotCarryTable {
   public static void setUp() throws Exception {
     Configuration c = UTIL.getConfiguration();
     // We use local filesystem.  Set it so it writes into the testdir.
-    FSUtils.setRootDir(c, UTIL.getDataTestDir());
+    CommonFSUtils.setRootDir(c, UTIL.getDataTestDir());
     UTIL.startMiniZKCluster();
     master = new HMaster(UTIL.getConfiguration());
     master.start();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/MockMasterServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/MockMasterServices.java
index 7d62836..89c422d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/MockMasterServices.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/MockMasterServices.java
@@ -56,7 +56,7 @@ import org.apache.hadoop.hbase.procedure2.store.NoopProcedureStore;
 import org.apache.hadoop.hbase.procedure2.store.ProcedureStore;
 import org.apache.hadoop.hbase.procedure2.store.ProcedureStore.ProcedureStoreListener;
 import org.apache.hadoop.hbase.security.Superusers;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.zookeeper.KeeperException;
 import org.mockito.Mockito;
 import org.mockito.invocation.InvocationOnMock;
@@ -149,8 +149,8 @@ public class MockMasterServices extends MockNoopMasterServices {
           Mockito.mock(AdminProtos.AdminService.BlockingInterface.class), ri, MOCK_MASTER_SERVERNAME,
           RegionInfoBuilder.FIRST_META_REGIONINFO);
     // Set hbase.rootdir into test dir.
-    Path rootdir = FSUtils.getRootDir(getConfiguration());
-    FSUtils.setRootDir(getConfiguration(), rootdir);
+    Path rootdir = CommonFSUtils.getRootDir(getConfiguration());
+    CommonFSUtils.setRootDir(getConfiguration(), rootdir);
   }
 
   public void start(final int numServes, final RSProcedureDispatcher remoteDispatcher)
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 61d29ec..2451f41 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
@@ -61,7 +61,7 @@ import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore;
 import org.apache.hadoop.hbase.regionserver.RegionServerAbortedException;
 import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.ipc.RemoteException;
 import org.junit.After;
 import org.junit.Before;
@@ -139,7 +139,7 @@ public abstract class TestAssignmentManagerBase {
   }
 
   protected void setupConfiguration(Configuration conf) throws Exception {
-    FSUtils.setRootDir(conf, util.getDataTestDir());
+    CommonFSUtils.setRootDir(conf, util.getDataTestDir());
     conf.setBoolean(WALProcedureStore.USE_HSYNC_CONF_KEY, false);
     conf.setInt(WALProcedureStore.SYNC_WAIT_MSEC_CONF_KEY, 10);
     conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, PROC_NTHREADS);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestHbckChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestHbckChore.java
index ea70508..b420f67 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestHbckChore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestHbckChore.java
@@ -25,7 +25,6 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.Future;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HRegionInfo;
@@ -40,6 +39,7 @@ import org.apache.hadoop.hbase.regionserver.HRegion;
 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.CommonFSUtils;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.Pair;
 import org.junit.Before;
@@ -209,7 +209,7 @@ public class TestHbckChore extends TestAssignmentManagerBase {
     hbckChore.choreForTesting();
     assertEquals(0, hbckChore.getOrphanRegionsOnFS().size());
 
-    HRegion.createRegionDir(conf, regionInfo, FSUtils.getRootDir(conf));
+    HRegion.createRegionDir(conf, regionInfo, CommonFSUtils.getRootDir(conf));
     hbckChore.choreForTesting();
     assertEquals(1, hbckChore.getOrphanRegionsOnFS().size());
     assertTrue(hbckChore.getOrphanRegionsOnFS().containsKey(regionInfo.getEncodedName()));
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestCleanerChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestCleanerChore.java
index c8e1853..af1e85b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestCleanerChore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestCleanerChore.java
@@ -35,7 +35,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.Stoppable;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.StoppableImplementation;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -282,7 +282,7 @@ public class TestCleanerChore {
       @Override
       public Boolean answer(InvocationOnMock invocation) throws Throwable {
         fs.create(addedFile).close();
-        FSUtils.logFileSystemState(fs, testDir, LOG);
+        CommonFSUtils.logFileSystemState(fs, testDir, LOG);
         return (Boolean) invocation.callRealMethod();
       }
     }).when(spy).isFileDeletable(Mockito.any());
@@ -342,7 +342,7 @@ public class TestCleanerChore {
       @Override
       public Boolean answer(InvocationOnMock invocation) throws Throwable {
         fs.create(racyFile).close();
-        FSUtils.logFileSystemState(fs, testDir, LOG);
+        CommonFSUtils.logFileSystemState(fs, testDir, LOG);
         return (Boolean) invocation.callRealMethod();
       }
     }).when(spy).isFileDeletable(Mockito.any());
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java
index 55557a6..1d0890c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java
@@ -40,7 +40,7 @@ import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.io.HFileLink;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.HFileArchiveUtil;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.junit.AfterClass;
@@ -81,9 +81,9 @@ public class TestHFileLinkCleaner {
   @Test
   public void testHFileLinkCleaning() throws Exception {
     Configuration conf = TEST_UTIL.getConfiguration();
-    FSUtils.setRootDir(conf, TEST_UTIL.getDataTestDir());
+    CommonFSUtils.setRootDir(conf, TEST_UTIL.getDataTestDir());
     conf.set(HFileCleaner.MASTER_HFILE_CLEANER_PLUGINS, HFileLinkCleaner.class.getName());
-    Path rootDir = FSUtils.getRootDir(conf);
+    Path rootDir = CommonFSUtils.getRootDir(conf);
     FileSystem fs = FileSystem.get(conf);
 
     final TableName tableName = TableName.valueOf(name.getMethodName());
@@ -127,8 +127,8 @@ public class TestHFileLinkCleaner {
     assertTrue(fs.exists(hfilePath));
 
     // Link backref can be removed
-    fs.rename(FSUtils.getTableDir(rootDir, tableLinkName),
-        FSUtils.getTableDir(archiveDir, tableLinkName));
+    fs.rename(CommonFSUtils.getTableDir(rootDir, tableLinkName),
+      CommonFSUtils.getTableDir(archiveDir, tableLinkName));
     cleaner.chore();
     assertFalse("Link should be deleted", fs.exists(linkBackRef));
 
@@ -142,13 +142,15 @@ public class TestHFileLinkCleaner {
       Thread.sleep(ttl * 2);
       cleaner.chore();
     }
-    assertFalse("HFile should be deleted", fs.exists(FSUtils.getTableDir(archiveDir, tableName)));
-    assertFalse("Link should be deleted", fs.exists(FSUtils.getTableDir(archiveDir, tableLinkName)));
+    assertFalse("HFile should be deleted",
+      fs.exists(CommonFSUtils.getTableDir(archiveDir, tableName)));
+    assertFalse("Link should be deleted",
+      fs.exists(CommonFSUtils.getTableDir(archiveDir, tableLinkName)));
   }
 
-  private static Path getFamilyDirPath (final Path rootDir, final TableName table,
+  private static Path getFamilyDirPath(final Path rootDir, final TableName table,
     final String region, final String family) {
-    return new Path(new Path(FSUtils.getTableDir(rootDir, table), region), family);
+    return new Path(new Path(CommonFSUtils.getTableDir(rootDir, table), region), family);
   }
 
   static class DummyServer implements Server {
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 9c1d495..fd309ec 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
@@ -28,7 +28,6 @@ import java.util.Set;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.regex.Pattern;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -59,8 +58,8 @@ import org.apache.hadoop.hbase.snapshot.UnknownSnapshotException;
 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.CommonFSUtils;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
 import org.junit.After;
 import org.junit.AfterClass;
@@ -413,7 +412,7 @@ public class TestSnapshotFromMaster {
     admin.snapshot(snapshotName, TABLE_NAME);
 
     LOG.info("After snapshot File-System state");
-    FSUtils.logFileSystemState(fs, rootDir, LOG);
+    CommonFSUtils.logFileSystemState(fs, rootDir, LOG);
 
     // ensure we only have one snapshot
     SnapshotTestingUtils.assertOneSnapshotThatMatches(admin, snapshotNameBytes, TABLE_NAME);
@@ -445,13 +444,13 @@ public class TestSnapshotFromMaster {
     CompactedHFilesDischarger cleaner = new CompactedHFilesDischarger(100, null, hrs, false);
     cleaner.chore();
     LOG.info("After compaction File-System state");
-    FSUtils.logFileSystemState(fs, rootDir, LOG);
+    CommonFSUtils.logFileSystemState(fs, rootDir, LOG);
 
     // make sure the cleaner has run
     LOG.debug("Running hfile cleaners");
     ensureHFileCleanersRun();
     LOG.info("After cleaners File-System state: " + rootDir);
-    FSUtils.logFileSystemState(fs, rootDir, LOG);
+    CommonFSUtils.logFileSystemState(fs, rootDir, LOG);
 
     // get the snapshot files for the table
     Path snapshotTable = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotName, rootDir);
@@ -492,7 +491,7 @@ public class TestSnapshotFromMaster {
     LOG.debug("Running hfile cleaners");
     ensureHFileCleanersRun();
     LOG.info("After delete snapshot cleaners run File-System state");
-    FSUtils.logFileSystemState(fs, rootDir, LOG);
+    CommonFSUtils.logFileSystemState(fs, rootDir, LOG);
 
     archives = getHFiles(archiveDir, fs, TABLE_NAME);
     assertEquals("Still have some hfiles in the archive, when their snapshot has been deleted.", 0,
@@ -504,7 +503,7 @@ public class TestSnapshotFromMaster {
    * @throws IOException on expected failure
    */
   private final Collection<String> getHFiles(Path dir, FileSystem fs, TableName tableName) throws IOException {
-    Path tableDir = FSUtils.getTableDir(dir, tableName);
+    Path tableDir = CommonFSUtils.getTableDir(dir, tableName);
     return SnapshotTestingUtils.listHFileNames(fs, tableDir);
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java
index 98c3978..c60a604 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java
@@ -58,6 +58,7 @@ import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
 import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.MD5Hash;
 import org.apache.hadoop.hbase.util.ModifyRegionUtils;
@@ -181,9 +182,10 @@ public class MasterProcedureTestingUtility {
       final RegionInfo[] regions, boolean hasFamilyDirs, String... family) throws IOException {
     // check filesystem
     final FileSystem fs = master.getMasterFileSystem().getFileSystem();
-    final Path tableDir = FSUtils.getTableDir(master.getMasterFileSystem().getRootDir(), tableName);
+    final Path tableDir =
+      CommonFSUtils.getTableDir(master.getMasterFileSystem().getRootDir(), tableName);
     assertTrue(fs.exists(tableDir));
-    FSUtils.logFileSystemState(fs, tableDir, LOG);
+    CommonFSUtils.logFileSystemState(fs, tableDir, LOG);
     List<Path> unwantedRegionDirs = FSUtils.getRegionDirs(fs, tableDir);
     for (int i = 0; i < regions.length; ++i) {
       Path regionDir = new Path(tableDir, regions[i].getEncodedName());
@@ -225,7 +227,8 @@ public class MasterProcedureTestingUtility {
       final HMaster master, final TableName tableName) throws IOException {
     // check filesystem
     final FileSystem fs = master.getMasterFileSystem().getFileSystem();
-    final Path tableDir = FSUtils.getTableDir(master.getMasterFileSystem().getRootDir(), tableName);
+    final Path tableDir =
+      CommonFSUtils.getTableDir(master.getMasterFileSystem().getRootDir(), tableName);
     assertFalse(fs.exists(tableDir));
 
     // check meta
@@ -301,8 +304,9 @@ public class MasterProcedureTestingUtility {
 
     // verify fs
     final FileSystem fs = master.getMasterFileSystem().getFileSystem();
-    final Path tableDir = FSUtils.getTableDir(master.getMasterFileSystem().getRootDir(), tableName);
-    for (Path regionDir: FSUtils.getRegionDirs(fs, tableDir)) {
+    final Path tableDir =
+      CommonFSUtils.getTableDir(master.getMasterFileSystem().getRootDir(), tableName);
+    for (Path regionDir : FSUtils.getRegionDirs(fs, tableDir)) {
       final Path familyDir = new Path(regionDir, family);
       assertFalse(family + " family dir should not exist", fs.exists(familyDir));
     }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateTableProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateTableProcedure.java
index 6609e62..3b57ed5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateTableProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateTableProcedure.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
 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.CommonFSUtils;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.ModifyRegionUtils;
 import org.apache.hadoop.hbase.util.TableDescriptorChecker;
@@ -224,7 +225,7 @@ public class TestCreateTableProcedure extends TestTableDDLProcedureBase {
           Configuration conf = env.getMasterConfiguration();
           MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
           Path tempdir = mfs.getTempDir();
-          Path tableDir = FSUtils.getTableDir(tempdir, regionInfo.getTable());
+          Path tableDir = CommonFSUtils.getTableDir(tempdir, regionInfo.getTable());
           Path regionDir = FSUtils.getRegionDirFromTableDir(tableDir, regionInfo);
           FileSystem fs = FileSystem.get(conf);
           fs.mkdirs(regionDir);
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 fb360da..173aa7d 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
@@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.master.procedure;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
+
 import java.io.IOException;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -37,7 +38,7 @@ import org.apache.hadoop.hbase.client.Table;
 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.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.wal.WALSplitUtil;
 import org.junit.After;
 import org.junit.AfterClass;
@@ -115,7 +116,7 @@ public class TestDeleteColumnFamilyProcedureFromClient {
     }
 
     // 3 - Check if table exists in FS
-    Path tableDir = FSUtils.getTableDir(TEST_UTIL.getDefaultRootDirPath(), TABLENAME);
+    Path tableDir = CommonFSUtils.getTableDir(TEST_UTIL.getDefaultRootDirPath(), TABLENAME);
     assertTrue(fs.exists(tableDir));
 
     // 4 - Check if all the 3 column families exist in FS
@@ -198,7 +199,7 @@ public class TestDeleteColumnFamilyProcedureFromClient {
     assertTrue(foundCF);
 
     // 3 - Check if table exists in FS
-    Path tableDir = FSUtils.getTableDir(TEST_UTIL.getDefaultRootDirPath(), TABLENAME);
+    Path tableDir = CommonFSUtils.getTableDir(TEST_UTIL.getDefaultRootDirPath(), TABLENAME);
     assertTrue(fs.exists(tableDir));
 
     // 4 - Check if all the target column family exist in FS
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteTableProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteTableProcedure.java
index fa83a93..1dd7dc4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteTableProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteTableProcedure.java
@@ -26,7 +26,6 @@ import static org.junit.Assert.fail;
 
 import java.util.ArrayList;
 import java.util.List;
-
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
@@ -44,7 +43,7 @@ import org.apache.hadoop.hbase.regionserver.HRegion;
 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.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.HFileArchiveTestingUtil;
 import org.junit.ClassRule;
 import org.junit.Rule;
@@ -215,9 +214,9 @@ public class TestDeleteTableProcedure extends TestTableDDLProcedureBase {
 
     final MasterFileSystem masterFileSystem =
       UTIL.getMiniHBaseCluster().getMaster().getMasterFileSystem();
-    final Path tableDir = FSUtils.getTableDir(masterFileSystem.getRootDir(), tableName);
+    final Path tableDir = CommonFSUtils.getTableDir(masterFileSystem.getRootDir(), tableName);
     final Path tempDir = masterFileSystem.getTempDir();
-    final Path tempTableDir = FSUtils.getTableDir(tempDir, tableName);
+    final Path tempTableDir = CommonFSUtils.getTableDir(tempDir, tableName);
     final FileSystem fs = masterFileSystem.getFileSystem();
 
     // copy the table to the temporary directory to make sure the temp directory is not empty
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestIgnoreUnknownFamily.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestIgnoreUnknownFamily.java
index 89ec68a..f173ea9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestIgnoreUnknownFamily.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestIgnoreUnknownFamily.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
 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.CommonFSUtils;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.junit.After;
 import org.junit.AfterClass;
@@ -80,7 +81,7 @@ public class TestIgnoreUnknownFamily {
   private void addStoreFileToKnownFamily(RegionInfo region) throws IOException {
     MasterFileSystem mfs = UTIL.getMiniHBaseCluster().getMaster().getMasterFileSystem();
     Path regionDir =
-        FSUtils.getRegionDirFromRootDir(FSUtils.getRootDir(mfs.getConfiguration()), region);
+      FSUtils.getRegionDirFromRootDir(CommonFSUtils.getRootDir(mfs.getConfiguration()), region);
     Path familyDir = new Path(regionDir, Bytes.toString(UNKNOWN_FAMILY));
     StoreFileWriter writer =
         new StoreFileWriter.Builder(mfs.getConfiguration(), mfs.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..1253710 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
@@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.ModifyRegionUtils;
 import org.junit.After;
 import org.junit.Before;
@@ -144,7 +144,7 @@ public class TestMasterFailoverWithProcedures {
     byte[][] splitKeys = null;
     RegionInfo[] regions = MasterProcedureTestingUtility.createTable(
         getMasterProcedureExecutor(), tableName, splitKeys, "f1", "f2");
-    Path tableDir = FSUtils.getTableDir(getRootDir(), tableName);
+    Path tableDir = CommonFSUtils.getTableDir(getRootDir(), tableName);
     MasterProcedureTestingUtility.validateTableCreation(
         UTIL.getHBaseCluster().getMaster(), tableName, regions, "f1", "f2");
     UTIL.getAdmin().disableTable(tableName);
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 5c56448..b3ece4e 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
@@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.master.procedure;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
+
 import java.io.IOException;
 import java.util.Set;
 import org.apache.hadoop.fs.Path;
@@ -34,8 +35,8 @@ import org.apache.hadoop.hbase.master.MasterFileSystem;
 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.CommonFSUtils;
 import org.apache.hadoop.hbase.util.FSTableDescriptors;
-import org.apache.hadoop.hbase.util.FSUtils;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.Before;
@@ -276,7 +277,7 @@ public class TestTableDescriptorModificationFromClient {
 
     // Verify descriptor from HDFS
     MasterFileSystem mfs = TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterFileSystem();
-    Path tableDir = FSUtils.getTableDir(mfs.getRootDir(), tableName);
+    Path tableDir = CommonFSUtils.getTableDir(mfs.getRootDir(), tableName);
     TableDescriptor td =
         FSTableDescriptors.getTableDescriptorFromFs(mfs.getFileSystem(), tableDir);
     verifyTableDescriptor(td, tableName, families);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTruncateTableProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTruncateTableProcedure.java
index d4bd227..95468d4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTruncateTableProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTruncateTableProcedure.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.master.procedure;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
+
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.stream.Collectors;
@@ -42,6 +43,7 @@ import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.ModifyRegionUtils;
 import org.junit.ClassRule;
@@ -51,6 +53,7 @@ import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
 
 @Category({ MasterTests.class, LargeTests.class })
@@ -272,7 +275,7 @@ public class TestTruncateTableProcedure extends TestTableDDLProcedureBase {
           Configuration conf = env.getMasterConfiguration();
           MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
           Path tempdir = mfs.getTempDir();
-          Path tableDir = FSUtils.getTableDir(tempdir, regionInfo.getTable());
+          Path tableDir = CommonFSUtils.getTableDir(tempdir, regionInfo.getTable());
           Path regionDir = FSUtils.getRegionDirFromTableDir(tableDir, regionInfo);
           FileSystem fs = FileSystem.get(conf);
           fs.mkdirs(regionDir);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotFileCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotFileCache.java
index 0dcb50b..0a8ddde 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotFileCache.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotFileCache.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.master.snapshot;
 
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
+
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -35,7 +36,7 @@ import org.apache.hadoop.hbase.snapshot.SnapshotReferenceUtil;
 import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils.SnapshotMock;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -179,16 +180,16 @@ public class TestSnapshotFileCache {
       assertFalse("Cache didn't find " + path, contains(getNonSnapshotFiles(cache, path), path));
     }
 
-    FSUtils.logFileSystemState(fs, rootDir, LOG);
+    CommonFSUtils.logFileSystemState(fs, rootDir, LOG);
     if (removeOnExit) {
       LOG.debug("Deleting snapshot.");
       fs.delete(builder.getSnapshotsDir(), true);
-      FSUtils.logFileSystemState(fs, rootDir, LOG);
+      CommonFSUtils.logFileSystemState(fs, rootDir, LOG);
 
       // then trigger a refresh
       cache.triggerCacheRefreshForTesting();
       // and not it shouldn't find those files
-      for (Path filePath: files) {
+      for (Path filePath : files) {
         assertTrue("Cache found '" + filePath + "', but it shouldn't have.",
           contains(getNonSnapshotFiles(cache, filePath), filePath));
 
@@ -207,9 +208,8 @@ public class TestSnapshotFileCache {
   }
 
   private static Iterable<FileStatus> getNonSnapshotFiles(SnapshotFileCache cache, Path storeFile)
-      throws IOException {
+    throws IOException {
     return cache.getUnreferencedFiles(
-        Arrays.asList(FSUtils.listStatus(fs, storeFile.getParent())), null
-    );
+      Arrays.asList(CommonFSUtils.listStatus(fs, storeFile.getParent())), null);
   }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotHFileCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotHFileCleaner.java
index 76c2a4b..a74d2c5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotHFileCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotHFileCleaner.java
@@ -36,7 +36,7 @@ import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
@@ -44,8 +44,6 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
  * Test that the snapshot hfile cleaner finds hfiles referenced in a snapshot
@@ -57,7 +55,6 @@ public class TestSnapshotHFileCleaner {
   public static final HBaseClassTestRule CLASS_RULE =
       HBaseClassTestRule.forClass(TestSnapshotHFileCleaner.class);
 
-  private static final Logger LOG = LoggerFactory.getLogger(TestSnapshotFileCache.class);
   private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
   private static final String TABLE_NAME_STR = "testSnapshotManifest";
   private static final String SNAPSHOT_NAME_STR = "testSnapshotManifest-snapshot";
@@ -73,7 +70,7 @@ public class TestSnapshotHFileCleaner {
   @BeforeClass
   public static void setup() throws Exception {
     Configuration conf = TEST_UTIL.getConfiguration();
-    rootDir = FSUtils.getRootDir(conf);
+    rootDir = CommonFSUtils.getRootDir(conf);
     fs = FileSystem.get(conf);
   }
 
@@ -87,8 +84,8 @@ public class TestSnapshotHFileCleaner {
   @Test
   public void testFindsSnapshotFilesWhenCleaning() throws IOException {
     Configuration conf = TEST_UTIL.getConfiguration();
-    FSUtils.setRootDir(conf, TEST_UTIL.getDataTestDir());
-    Path rootDir = FSUtils.getRootDir(conf);
+    CommonFSUtils.setRootDir(conf, TEST_UTIL.getDataTestDir());
+    Path rootDir = CommonFSUtils.getRootDir(conf);
     Path archivedHfileDir = new Path(TEST_UTIL.getDataTestDir(), HConstants.HFILE_ARCHIVE_DIRECTORY);
 
     FileSystem fs = FileSystem.get(conf);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileLink.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileLink.java
index 57bcf1f..a60153e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileLink.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileLink.java
@@ -26,7 +26,7 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.io.HFileLink;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.HFileArchiveUtil;
 import org.junit.Assert;
 import org.junit.ClassRule;
@@ -50,9 +50,9 @@ public class TestMobFileLink {
     final TableName tableName = TableName.valueOf(name.getMethodName());
     Configuration conf = HBaseConfiguration.create();
     FileSystem fs = FileSystem.get(conf);
-    Path rootDir = FSUtils.getRootDir(conf);
-    Path tableDir = FSUtils.getTableDir(rootDir, tableName);
-    Path archiveDir = FSUtils.getTableDir(HFileArchiveUtil.getArchivePath(conf), tableName);
+    Path rootDir = CommonFSUtils.getRootDir(conf);
+    Path tableDir = CommonFSUtils.getTableDir(rootDir, tableName);
+    Path archiveDir = CommonFSUtils.getTableDir(HFileArchiveUtil.getArchivePath(conf), tableName);
     String fileName = "mobFile";
     String encodedRegionName = MobUtils.getMobRegionInfo(tableName).getEncodedName();
     String columnFamily = "columnFamily";
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactor.java
index 2cf741e..d4c70ab 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactor.java
@@ -73,7 +73,7 @@ import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
 import org.apache.hadoop.hbase.regionserver.StoreScanner;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.junit.AfterClass;
@@ -132,7 +132,7 @@ public class TestPartitionedMobCompactor {
 
   private void init(String tableName) throws Exception {
     fs = FileSystem.get(conf);
-    Path testDir = FSUtils.getRootDir(conf);
+    Path testDir = CommonFSUtils.getRootDir(conf);
     Path mobTestDir = new Path(testDir, MobConstants.MOB_DIR_NAME);
     basePath = new Path(new Path(mobTestDir, tableName), family);
     mobSuffix = TEST_UTIL.getRandomUUID().toString().replaceAll("-", "");
@@ -369,7 +369,7 @@ public class TestPartitionedMobCompactor {
     String tableName = name.getMethodName();
     fs = FileSystem.get(conf);
     FaultyDistributedFileSystem faultyFs = (FaultyDistributedFileSystem)fs;
-    Path testDir = FSUtils.getRootDir(conf);
+    Path testDir = CommonFSUtils.getRootDir(conf);
     Path mobTestDir = new Path(testDir, MobConstants.MOB_DIR_NAME);
     basePath = new Path(new Path(mobTestDir, tableName), family);
 
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 e7f2204..4324950 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
@@ -81,7 +81,7 @@ import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
 import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.zookeeper.KeeperException;
 import org.junit.After;
 import org.junit.AfterClass;
@@ -205,7 +205,7 @@ public class TestNamespaceAuditor {
       exceptionCaught = true;
     } finally {
       assertTrue(exceptionCaught);
-      assertFalse(fs.exists(FSUtils.getNamespaceDir(rootDir, nspDesc.getName())));
+      assertFalse(fs.exists(CommonFSUtils.getNamespaceDir(rootDir, nspDesc.getName())));
     }
     nspDesc =
         NamespaceDescriptor.create(prefix + "vq2")
@@ -218,7 +218,7 @@ public class TestNamespaceAuditor {
       exceptionCaught = true;
     } finally {
       assertTrue(exceptionCaught);
-      assertFalse(fs.exists(FSUtils.getNamespaceDir(rootDir, nspDesc.getName())));
+      assertFalse(fs.exists(CommonFSUtils.getNamespaceDir(rootDir, nspDesc.getName())));
     }
     nspDesc =
         NamespaceDescriptor.create(prefix + "vq3")
@@ -231,7 +231,7 @@ public class TestNamespaceAuditor {
       exceptionCaught = true;
     } finally {
       assertTrue(exceptionCaught);
-      assertFalse(fs.exists(FSUtils.getNamespaceDir(rootDir, nspDesc.getName())));
+      assertFalse(fs.exists(CommonFSUtils.getNamespaceDir(rootDir, nspDesc.getName())));
     }
     nspDesc =
         NamespaceDescriptor.create(prefix + "vq4")
@@ -244,7 +244,7 @@ public class TestNamespaceAuditor {
       exceptionCaught = true;
     } finally {
       assertTrue(exceptionCaught);
-      assertFalse(fs.exists(FSUtils.getNamespaceDir(rootDir, nspDesc.getName())));
+      assertFalse(fs.exists(CommonFSUtils.getNamespaceDir(rootDir, nspDesc.getName())));
     }
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestFileArchiverNotifierImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestFileArchiverNotifierImpl.java
index e139e4f..c5750bb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestFileArchiverNotifierImpl.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestFileArchiverNotifierImpl.java
@@ -29,7 +29,6 @@ import java.util.List;
 import java.util.Map.Entry;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicLong;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -54,7 +53,7 @@ import org.apache.hadoop.hbase.quotas.FileArchiverNotifierImpl.SnapshotWithSize;
 import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
 import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
@@ -289,7 +288,7 @@ public class TestFileArchiverNotifierImpl {
   private Set<String> getFilesReferencedBySnapshot(String snapshotName) throws IOException {
     HashSet<String> files = new HashSet<>();
     Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(
-        snapshotName, FSUtils.getRootDir(conf));
+        snapshotName, CommonFSUtils.getRootDir(conf));
     SnapshotProtos.SnapshotDescription sd = SnapshotDescriptionUtils.readSnapshotInfo(
         fs, snapshotDir);
     SnapshotManifest manifest = SnapshotManifest.open(conf, fs, snapshotDir, sd);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoadReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoadReplication.java
index 33ab4e3..0212655 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoadReplication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoadReplication.java
@@ -21,6 +21,7 @@ import static org.apache.hadoop.hbase.HConstants.REPLICATION_CLUSTER_ID;
 import static org.apache.hadoop.hbase.HConstants.REPLICATION_CONF_DIR;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
+
 import java.io.File;
 import java.io.FileOutputStream;
 import java.io.IOException;
@@ -76,7 +77,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
 import org.apache.hadoop.hbase.tool.BulkLoadHFilesTool;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.junit.After;
@@ -349,7 +350,7 @@ public class TestBulkLoadReplication extends TestReplicationBase {
   }
 
   private Path createMobFiles(HBaseTestingUtility util) throws IOException {
-    Path testDir = FSUtils.getRootDir(util.getConfiguration());
+    Path testDir = CommonFSUtils.getRootDir(util.getConfiguration());
     Path mobTestDir = new Path(testDir, MobConstants.MOB_DIR_NAME);
     Path basePath = new Path(new Path(mobTestDir, tableName.getNameAsString()), "f");
     HFileContext meta = new HFileContextBuilder().withBlockSize(8 * 1024).build();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java
index aec15f7..e2eefa4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hbase.regionserver;
 
 import static org.junit.Assert.assertTrue;
+
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -50,7 +51,7 @@ import org.apache.hadoop.hbase.io.hfile.RandomKeyValueUtil;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
 import org.apache.hadoop.hbase.wal.WALFactory;
 import org.junit.After;
@@ -173,7 +174,8 @@ public class TestCacheOnWriteInSchema {
 
     // Create a store based on the schema
     String id = TestCacheOnWriteInSchema.class.getName();
-    Path logdir = new Path(FSUtils.getRootDir(conf), AbstractFSWALProvider.getWALDirectoryName(id));
+    Path logdir =
+      new Path(CommonFSUtils.getRootDir(conf), AbstractFSWALProvider.getWALDirectoryName(id));
     fs.delete(logdir, true);
 
     RegionInfo info = RegionInfoBuilder.newBuilder(htd.getTableName()).build();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCleanupMetaWAL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCleanupMetaWAL.java
index a5c2987..fd1e533 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCleanupMetaWAL.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCleanupMetaWAL.java
@@ -17,6 +17,9 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
+import static org.apache.hadoop.hbase.wal.AbstractFSWALProvider.SPLITTING_EXT;
+import static org.junit.Assert.fail;
+
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
@@ -27,7 +30,7 @@ import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.master.MasterFileSystem;
 import org.apache.hadoop.hbase.master.procedure.ServerCrashProcedure;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
@@ -36,9 +39,6 @@ import org.junit.experimental.categories.Category;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.apache.hadoop.hbase.wal.AbstractFSWALProvider.SPLITTING_EXT;
-import static org.junit.Assert.fail;
-
 @Category(MediumTests.class)
 public class TestCleanupMetaWAL {
   private static final Logger LOG = LoggerFactory.getLogger(TestCleanupMetaWAL.class);
@@ -75,7 +75,7 @@ public class TestCleanupMetaWAL {
     LOG.info("DONE WAITING");
     MasterFileSystem fs = TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterFileSystem();
     Path walPath = new Path(fs.getWALRootDir(), HConstants.HREGION_LOGDIR_NAME);
-    for (FileStatus status : FSUtils.listStatus(fs.getFileSystem(), walPath)) {
+    for (FileStatus status : CommonFSUtils.listStatus(fs.getFileSystem(), walPath)) {
       if (status.getPath().toString().contains(SPLITTING_EXT)) {
         fail("Should not have splitting wal dir here:" + status);
       }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestClusterId.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestClusterId.java
index 189fd2e..16799a3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestClusterId.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestClusterId.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseCommonTestingUtility;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.StartMiniClusterOption;
@@ -32,7 +33,7 @@ import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.LoadBalancer;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.JVMClusterUtil;
 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
 import org.junit.After;
@@ -99,13 +100,13 @@ public class TestClusterId {
     TEST_UTIL.startMiniZKCluster();
     TEST_UTIL.startMiniDFSCluster(1);
     TEST_UTIL.createRootDir();
-    Path rootDir = FSUtils.getRootDir(TEST_UTIL.getConfiguration());
+    Path rootDir = CommonFSUtils.getRootDir(TEST_UTIL.getConfiguration());
     FileSystem fs = rootDir.getFileSystem(TEST_UTIL.getConfiguration());
     Path filePath = new Path(rootDir, HConstants.CLUSTER_ID_FILE_NAME);
     FSDataOutputStream s = null;
     try {
       s = fs.create(filePath);
-      s.writeUTF(TEST_UTIL.getRandomUUID().toString());
+      s.writeUTF(HBaseCommonTestingUtility.getRandomUUID().toString());
     } finally {
       if (s != null) {
         s.close();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSplitThread.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSplitThread.java
index f7f0ca0..bb52e19 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSplitThread.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSplitThread.java
@@ -33,7 +33,7 @@ import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
 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.CommonFSUtils;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Assert;
@@ -170,7 +170,7 @@ public class TestCompactSplitThread {
     }
 
     // Make sure that store file number is greater than blockingStoreFiles + 1
-    Path tableDir = FSUtils.getTableDir(rootDir, tableName);
+    Path tableDir = CommonFSUtils.getTableDir(rootDir, tableName);
     Collection<String> hfiles =  SnapshotTestingUtils.listHFileNames(fs, tableDir);
     assert(hfiles.size() > blockingStoreFiles + 1);
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java
index 8e2f7bd..18e84c9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.regionserver;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.mockito.Mockito.mock;
+
 import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.util.ArrayList;
@@ -43,7 +44,7 @@ import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.wal.WALFactory;
 import org.junit.After;
 import org.junit.Before;
@@ -77,7 +78,7 @@ public class TestCompactionArchiveConcurrentClose {
   public void setup() throws Exception {
     testUtil = HBaseTestingUtility.createLocalHTU();
     testDir = testUtil.getDataTestDir("TestStoreFileRefresherChore");
-    FSUtils.setRootDir(testUtil.getConfiguration(), testDir);
+    CommonFSUtils.setRootDir(testUtil.getConfiguration(), testDir);
   }
 
   @After
@@ -167,13 +168,13 @@ public class TestCompactionArchiveConcurrentClose {
 
   private HRegion initHRegion(TableDescriptor htd, RegionInfo info) throws IOException {
     Configuration conf = testUtil.getConfiguration();
-    Path tableDir = FSUtils.getTableDir(testDir, htd.getTableName());
+    Path tableDir = CommonFSUtils.getTableDir(testDir, htd.getTableName());
 
     HRegionFileSystem fs =
         new WaitingHRegionFileSystem(conf, tableDir.getFileSystem(conf), tableDir, info);
     ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     final Configuration walConf = new Configuration(conf);
-    FSUtils.setRootDir(walConf, tableDir);
+    CommonFSUtils.setRootDir(walConf, tableDir);
     final WALFactory wals = new WALFactory(walConf, "log_" + info.getEncodedName());
     HRegion region = new HRegion(fs, wals.getWAL(info), conf, htd, null);
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveIOException.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveIOException.java
index 1199c6f..8f40ba0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveIOException.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveIOException.java
@@ -48,7 +48,7 @@ import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 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.CommonFSUtils;
 import org.apache.hadoop.hbase.wal.WALFactory;
 import org.junit.After;
 import org.junit.Before;
@@ -85,7 +85,7 @@ public class TestCompactionArchiveIOException {
     testUtil = new HBaseTestingUtility();
     testUtil.startMiniDFSCluster(1);
     testDir = testUtil.getDataTestDirOnTestFS();
-    FSUtils.setRootDir(testUtil.getConfiguration(), testDir);
+    CommonFSUtils.setRootDir(testUtil.getConfiguration(), testDir);
   }
 
   @After
@@ -182,7 +182,7 @@ public class TestCompactionArchiveIOException {
   private HRegion initHRegion(TableDescriptor htd, RegionInfo info) throws IOException {
     Configuration conf = testUtil.getConfiguration();
     ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
-    Path tableDir = FSUtils.getTableDir(testDir, htd.getTableName());
+    Path tableDir = CommonFSUtils.getTableDir(testDir, htd.getTableName());
     Path regionDir = new Path(tableDir, info.getEncodedName());
     Path storeDir = new Path(regionDir, htd.getColumnFamilies()[0].getNameAsString());
 
@@ -195,7 +195,7 @@ public class TestCompactionArchiveIOException {
 
     HRegionFileSystem fs = new HRegionFileSystem(conf, errFS, tableDir, info);
     final Configuration walConf = new Configuration(conf);
-    FSUtils.setRootDir(walConf, tableDir);
+    CommonFSUtils.setRootDir(walConf, tableDir);
     final WALFactory wals = new WALFactory(walConf, "log_" + info.getEncodedName());
     HRegion region = new HRegion(fs, wals.getWAL(info), conf, htd, null);
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java
index d48e1e1..e3bc07f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java
@@ -22,7 +22,6 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -37,12 +36,13 @@ import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequestImpl;
 import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy;
 import org.apache.hadoop.hbase.regionserver.wal.FSHLog;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 
 public class TestCompactionPolicy {
@@ -103,9 +103,8 @@ public class TestCompactionPolicy {
     ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     region = HRegion.createHRegion(info, basedir, conf, htd, hlog);
     region.close();
-    Path tableDir = FSUtils.getTableDir(basedir, htd.getTableName());
+    Path tableDir = CommonFSUtils.getTableDir(basedir, htd.getTableName());
     region = new HRegion(tableDir, hlog, fs, conf, info, htd, null);
-
     store = new HStore(region, hcd, conf, false);
 
     TEST_FILE = region.getRegionFileSystem().createTempName();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDeleteMobTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDeleteMobTable.java
index b327668..122cde9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDeleteMobTable.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDeleteMobTable.java
@@ -35,8 +35,8 @@ import org.apache.hadoop.hbase.mob.MobConstants;
 import org.apache.hadoop.hbase.mob.MobUtils;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.HFileArchiveUtil;
 import org.junit.AfterClass;
 import org.junit.Assert;
@@ -215,7 +215,8 @@ public class TestDeleteMobTable {
 
   private boolean mobTableDirExist(TableName tn) throws IOException {
     FileSystem fs = TEST_UTIL.getTestFileSystem();
-    Path tableDir = FSUtils.getTableDir(MobUtils.getMobHome(TEST_UTIL.getConfiguration()), tn);
+    Path tableDir =
+      CommonFSUtils.getTableDir(MobUtils.getMobHome(TEST_UTIL.getConfiguration()), tn);
     return fs.exists(tableDir);
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java
index 152ea87..9340f37 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java
@@ -32,7 +32,6 @@ import java.util.NavigableSet;
 import java.util.Optional;
 import java.util.concurrent.ConcurrentSkipListSet;
 import javax.crypto.spec.SecretKeySpec;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -70,7 +69,7 @@ import org.apache.hadoop.hbase.security.EncryptionUtil;
 import org.apache.hadoop.hbase.security.User;
 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.CommonFSUtils;
 import org.apache.hadoop.hbase.wal.WALFactory;
 import org.junit.Assert;
 import org.junit.Before;
@@ -153,7 +152,7 @@ public class TestHMobStore {
 
     //Setting up tje Region and Store
     Path basedir = new Path(DIR + methodName);
-    Path tableDir = FSUtils.getTableDir(basedir, td.getTableName());
+    Path tableDir = CommonFSUtils.getTableDir(basedir, td.getTableName());
     String logName = "logs";
     Path logdir = new Path(basedir, logName);
     FileSystem fs = FileSystem.get(conf);
@@ -162,7 +161,7 @@ public class TestHMobStore {
     RegionInfo info = RegionInfoBuilder.newBuilder(td.getTableName()).build();
     ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     final Configuration walConf = new Configuration(conf);
-    FSUtils.setRootDir(walConf, basedir);
+    CommonFSUtils.setRootDir(walConf, basedir);
     final WALFactory wals = new WALFactory(walConf, methodName);
     region = new HRegion(tableDir, wals.getWAL(info), fs, conf, info, td, null);
     region.setMobFileCache(new MobFileCache(conf));
@@ -174,7 +173,7 @@ public class TestHMobStore {
 
   private void init(Configuration conf, ColumnFamilyDescriptor cfd)
       throws IOException {
-    Path basedir = FSUtils.getRootDir(conf);
+    Path basedir = CommonFSUtils.getRootDir(conf);
     fs = FileSystem.get(conf);
     Path homePath = new Path(basedir, Bytes.toString(family) + Path.SEPARATOR
         + Bytes.toString(family));
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
index 5c6f117..3448eb7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
@@ -152,7 +152,6 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper;
-import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.HFileArchiveUtil;
 import org.apache.hadoop.hbase.util.IncrementingEnvironmentEdge;
 import org.apache.hadoop.hbase.util.ManualEnvironmentEdge;
@@ -393,7 +392,7 @@ public class TestHRegion {
       Configuration conf, TableName tableName) throws IOException {
     final Path logDir = TEST_UTIL.getDataTestDirOnTestFS(callingMethod + ".log");
     final Configuration walConf = new Configuration(conf);
-    FSUtils.setRootDir(walConf, logDir);
+    CommonFSUtils.setRootDir(walConf, logDir);
     return new WALFactory(walConf, callingMethod)
         .getWAL(RegionInfoBuilder.newBuilder(tableName).build());
   }
@@ -976,7 +975,7 @@ public class TestHRegion {
 
       // now find the compacted file, and manually add it to the recovered edits
       Path tmpDir = new Path(region.getRegionFileSystem().getTempDir(), Bytes.toString(family));
-      FileStatus[] files = FSUtils.listStatus(fs, tmpDir);
+      FileStatus[] files = CommonFSUtils.listStatus(fs, tmpDir);
       String errorMsg = "Expected to find 1 file in the region temp directory "
           + "from the compaction, could not find any";
       assertNotNull(errorMsg, files);
@@ -1031,7 +1030,7 @@ public class TestHRegion {
       if (!mismatchedRegionName) {
         assertEquals(1, region.getStore(family).getStorefilesCount());
       }
-      files = FSUtils.listStatus(fs, tmpDir);
+      files = CommonFSUtils.listStatus(fs, tmpDir);
       assertTrue("Expected to find 0 files inside " + tmpDir, files == null || files.length == 0);
 
       for (long i = minSeqId; i < maxSeqId; i++) {
@@ -1054,7 +1053,7 @@ public class TestHRegion {
     byte[] family = Bytes.toBytes("family");
     Path logDir = TEST_UTIL.getDataTestDirOnTestFS(method + ".log");
     final Configuration walConf = new Configuration(TEST_UTIL.getConfiguration());
-    FSUtils.setRootDir(walConf, logDir);
+    CommonFSUtils.setRootDir(walConf, logDir);
     final WALFactory wals = new WALFactory(walConf, method);
     final WAL wal = wals.getWAL(RegionInfoBuilder.newBuilder(tableName).build());
 
@@ -1208,7 +1207,7 @@ public class TestHRegion {
     Path logDir = TEST_UTIL.getDataTestDirOnTestFS(method + "log");
 
     final Configuration walConf = new Configuration(TEST_UTIL.getConfiguration());
-    FSUtils.setRootDir(walConf, logDir);
+    CommonFSUtils.setRootDir(walConf, logDir);
     // Make up a WAL that we can manipulate at append time.
     class FailAppendFlushMarkerWAL extends FSHLog {
       volatile FlushAction [] flushActions = null;
@@ -1255,9 +1254,8 @@ public class TestHRegion {
         };
       }
     }
-    FailAppendFlushMarkerWAL wal =
-      new FailAppendFlushMarkerWAL(FileSystem.get(walConf), FSUtils.getRootDir(walConf),
-        method, walConf);
+    FailAppendFlushMarkerWAL wal = new FailAppendFlushMarkerWAL(FileSystem.get(walConf),
+      CommonFSUtils.getRootDir(walConf), method, walConf);
     wal.init();
     this.region = initHRegion(tableName, HConstants.EMPTY_START_ROW,
       HConstants.EMPTY_END_ROW, false, Durability.USE_DEFAULT, wal, family);
@@ -1286,9 +1284,9 @@ public class TestHRegion {
     wal.close();
 
     // 2. Test case where START_FLUSH succeeds but COMMIT_FLUSH will throw exception
-    wal.flushActions = new FlushAction [] {FlushAction.COMMIT_FLUSH};
-    wal = new FailAppendFlushMarkerWAL(FileSystem.get(walConf), FSUtils.getRootDir(walConf),
-          method, walConf);
+    wal.flushActions = new FlushAction[] { FlushAction.COMMIT_FLUSH };
+    wal = new FailAppendFlushMarkerWAL(FileSystem.get(walConf), CommonFSUtils.getRootDir(walConf),
+      method, walConf);
     wal.init();
     this.region = initHRegion(tableName, HConstants.EMPTY_START_ROW,
       HConstants.EMPTY_END_ROW, false, Durability.USE_DEFAULT, wal, family);
@@ -4755,7 +4753,7 @@ public class TestHRegion {
     byte[] family = Bytes.toBytes("family");
     Path logDir = new Path(new Path(dir + method), "log");
     final Configuration walConf = new Configuration(conf);
-    FSUtils.setRootDir(walConf, logDir);
+    CommonFSUtils.setRootDir(walConf, logDir);
     // XXX: The spied AsyncFSWAL can not work properly because of a Mockito defect that can not
     // deal with classes which have a field of an inner class. See discussions in HBASE-15536.
     walConf.set(WALFactory.WAL_PROVIDER, "filesystem");
@@ -4805,7 +4803,7 @@ public class TestHRegion {
     // create a primary region, load some data and flush
     // create a secondary region, and do a get against that
     Path rootDir = new Path(dir + name.getMethodName());
-    FSUtils.setRootDir(TEST_UTIL.getConfiguration(), rootDir);
+    CommonFSUtils.setRootDir(TEST_UTIL.getConfiguration(), rootDir);
 
     byte[][] families = new byte[][] {
         Bytes.toBytes("cf1"), Bytes.toBytes("cf2"), Bytes.toBytes("cf3")
@@ -4855,7 +4853,7 @@ public class TestHRegion {
     // create a primary region, load some data and flush
     // create a secondary region, and do a put against that
     Path rootDir = new Path(dir + name.getMethodName());
-    FSUtils.setRootDir(TEST_UTIL.getConfiguration(), rootDir);
+    CommonFSUtils.setRootDir(TEST_UTIL.getConfiguration(), rootDir);
 
     byte[][] families = new byte[][] {
         Bytes.toBytes("cf1"), Bytes.toBytes("cf2"), Bytes.toBytes("cf3")
@@ -4914,7 +4912,7 @@ public class TestHRegion {
   @Test
   public void testCompactionFromPrimary() throws IOException {
     Path rootDir = new Path(dir + name.getMethodName());
-    FSUtils.setRootDir(TEST_UTIL.getConfiguration(), rootDir);
+    CommonFSUtils.setRootDir(TEST_UTIL.getConfiguration(), rootDir);
 
     byte[][] families = new byte[][] {
         Bytes.toBytes("cf1"), Bytes.toBytes("cf2"), Bytes.toBytes("cf3")
@@ -6005,7 +6003,7 @@ public class TestHRegion {
   @Test
   public void testCloseRegionWrittenToWAL() throws Exception {
     Path rootDir = new Path(dir + name.getMethodName());
-    FSUtils.setRootDir(TEST_UTIL.getConfiguration(), rootDir);
+    CommonFSUtils.setRootDir(TEST_UTIL.getConfiguration(), rootDir);
 
     final ServerName serverName = ServerName.valueOf("testCloseRegionWrittenToWAL", 100, 42);
     final RegionServerServices rss = spy(TEST_UTIL.createMockRegionServerService(serverName));
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionFileSystem.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionFileSystem.java
index e7e636f..39c525d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionFileSystem.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionFileSystem.java
@@ -47,6 +47,7 @@ import org.apache.hadoop.hbase.fs.HFileSystem;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.util.Progressable;
 import org.junit.ClassRule;
@@ -146,13 +147,13 @@ public class TestHRegionFileSystem {
       // there should be 3 files in store dir
       FileSystem fs = TEST_UTIL.getDFSCluster().getFileSystem();
       Path storePath = regionFs.getStoreDir(Bytes.toString(FAMILIES[0]));
-      FileStatus[] storeFiles = FSUtils.listStatus(fs, storePath);
+      FileStatus[] storeFiles = CommonFSUtils.listStatus(fs, storePath);
       assertNotNull(storeFiles);
       assertEquals(3, storeFiles.length);
       // store temp dir still exists but empty
       Path storeTempDir = new Path(regionFs.getTempDir(), Bytes.toString(FAMILIES[0]));
       assertTrue(fs.exists(storeTempDir));
-      FileStatus[] tempFiles = FSUtils.listStatus(fs, storeTempDir);
+      FileStatus[] tempFiles = CommonFSUtils.listStatus(fs, storeTempDir);
       assertNull(tempFiles);
       // storage policy of cf temp dir and 3 store files should be ONE_SSD
       assertEquals("ONE_SSD",
@@ -182,7 +183,7 @@ public class TestHRegionFileSystem {
 
   private HRegionFileSystem getHRegionFS(HTable table, Configuration conf) throws IOException {
     FileSystem fs = TEST_UTIL.getDFSCluster().getFileSystem();
-    Path tableDir = FSUtils.getTableDir(TEST_UTIL.getDefaultRootDirPath(), table.getName());
+    Path tableDir = CommonFSUtils.getTableDir(TEST_UTIL.getDefaultRootDirPath(), table.getName());
     List<Path> regionDirs = FSUtils.getRegionDirs(fs, tableDir);
     assertEquals(1, regionDirs.size());
     List<Path> familyDirs = FSUtils.getFamilyDirs(fs, regionDirs.get(0));
@@ -201,7 +202,7 @@ public class TestHRegionFileSystem {
     // Create a Region
     RegionInfo hri = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName())).build();
     HRegionFileSystem regionFs = HRegionFileSystem.createRegionOnFileSystem(conf, fs,
-        FSUtils.getTableDir(rootDir, hri.getTable()), hri);
+      CommonFSUtils.getTableDir(rootDir, hri.getTable()), hri);
 
     // Verify if the region is on disk
     Path regionDir = regionFs.getRegionDir();
@@ -213,12 +214,12 @@ public class TestHRegionFileSystem {
 
     // Open the region
     regionFs = HRegionFileSystem.openRegionFromFileSystem(conf, fs,
-        FSUtils.getTableDir(rootDir, hri.getTable()), hri, false);
+      CommonFSUtils.getTableDir(rootDir, hri.getTable()), hri, false);
     assertEquals(regionDir, regionFs.getRegionDir());
 
     // Delete the region
     HRegionFileSystem.deleteRegionFromFileSystem(conf, fs,
-        FSUtils.getTableDir(rootDir, hri.getTable()), hri);
+      CommonFSUtils.getTableDir(rootDir, hri.getTable()), hri);
     assertFalse("The region folder should be removed", fs.exists(regionDir));
 
     fs.delete(rootDir, true);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java
index 733618b..33d615b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java
@@ -99,9 +99,9 @@ import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper;
-import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.IncrementingEnvironmentEdge;
 import org.apache.hadoop.hbase.util.ManualEnvironmentEdge;
 import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
@@ -210,7 +210,7 @@ public class TestHStore {
       ColumnFamilyDescriptor hcd, MyStoreHook hook, boolean switchToPread) throws IOException {
     TableDescriptor htd = builder.setColumnFamily(hcd).build();
     Path basedir = new Path(DIR + methodName);
-    Path tableDir = FSUtils.getTableDir(basedir, htd.getTableName());
+    Path tableDir = CommonFSUtils.getTableDir(basedir, htd.getTableName());
     final Path logdir = new Path(basedir, AbstractFSWALProvider.getWALDirectoryName(methodName));
 
     FileSystem fs = FileSystem.get(conf);
@@ -220,7 +220,7 @@ public class TestHStore {
       MemStoreLABImpl.CHUNK_SIZE_DEFAULT, 1, 0, null);
     RegionInfo info = RegionInfoBuilder.newBuilder(htd.getTableName()).build();
     Configuration walConf = new Configuration(conf);
-    FSUtils.setRootDir(walConf, basedir);
+    CommonFSUtils.setRootDir(walConf, basedir);
     WALFactory wals = new WALFactory(walConf, methodName);
     region = new HRegion(new HRegionFileSystem(conf, fs, tableDir, info), wals.getWAL(info), conf,
         htd, null);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStoreFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStoreFile.java
index 8ee8ed3..b7d0db0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStoreFile.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStoreFile.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.regionserver;
 
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
+
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
@@ -69,10 +70,7 @@ import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.BloomFilterFactory;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ChecksumType;
-import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.hbase.thirdparty.com.google.common.base.Joiner;
-import org.apache.hbase.thirdparty.com.google.common.collect.Iterables;
-import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.ClassRule;
@@ -82,6 +80,10 @@ import org.mockito.Mockito;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hbase.thirdparty.com.google.common.base.Joiner;
+import org.apache.hbase.thirdparty.com.google.common.collect.Iterables;
+import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
+
 /**
  * Test HStoreFile
  */
@@ -262,9 +264,9 @@ public class TestHStoreFile extends HBaseTestCase {
     final HRegionInfo hri = new HRegionInfo(TableName.valueOf("testHFileLinkTb"));
     // force temp data in hbase/target/test-data instead of /tmp/hbase-xxxx/
     Configuration testConf = new Configuration(this.conf);
-    FSUtils.setRootDir(testConf, testDir);
+    CommonFSUtils.setRootDir(testConf, testDir);
     HRegionFileSystem regionFs = HRegionFileSystem.createRegionOnFileSystem(
-      testConf, fs, FSUtils.getTableDir(testDir, hri.getTable()), hri);
+      testConf, fs, CommonFSUtils.getTableDir(testDir, hri.getTable()), hri);
     HFileContext meta = new HFileContextBuilder().withBlockSize(8 * 1024).build();
 
     // Make a store file and write data to it.
@@ -304,12 +306,12 @@ public class TestHStoreFile extends HBaseTestCase {
   public void testReferenceToHFileLink() throws IOException {
     // force temp data in hbase/target/test-data instead of /tmp/hbase-xxxx/
     Configuration testConf = new Configuration(this.conf);
-    FSUtils.setRootDir(testConf, testDir);
+    CommonFSUtils.setRootDir(testConf, testDir);
 
     // adding legal table name chars to verify regex handles it.
     HRegionInfo hri = new HRegionInfo(TableName.valueOf("_original-evil-name"));
     HRegionFileSystem regionFs = HRegionFileSystem.createRegionOnFileSystem(
-      testConf, fs, FSUtils.getTableDir(testDir, hri.getTable()), hri);
+      testConf, fs, CommonFSUtils.getTableDir(testDir, hri.getTable()), hri);
 
     HFileContext meta = new HFileContextBuilder().withBlockSize(8 * 1024).build();
     // Make a store file and write data to it. <root>/<tablename>/<rgn>/<cf>/<file>
@@ -323,7 +325,7 @@ public class TestHStoreFile extends HBaseTestCase {
     // create link to store file. <root>/clone/region/<cf>/<hfile>-<region>-<table>
     HRegionInfo hriClone = new HRegionInfo(TableName.valueOf("clone"));
     HRegionFileSystem cloneRegionFs = HRegionFileSystem.createRegionOnFileSystem(
-      testConf, fs, FSUtils.getTableDir(testDir, hri.getTable()),
+      testConf, fs, CommonFSUtils.getTableDir(testDir, hri.getTable()),
         hriClone);
     Path dstPath = cloneRegionFs.getStoreDir(TEST_FAMILY);
     HFileLink.create(testConf, this.fs, dstPath, hri, storeFilePath.getName());
@@ -341,7 +343,7 @@ public class TestHStoreFile extends HBaseTestCase {
     Path pathB = splitStoreFile(cloneRegionFs, splitHriB, TEST_FAMILY, f, SPLITKEY, false);// bottom
     f.closeStoreFile(true);
     // OK test the thing
-    FSUtils.logFileSystemState(fs, testDir, LOG);
+    CommonFSUtils.logFileSystemState(fs, testDir, LOG);
 
     // There is a case where a file with the hfilelink pattern is actually a daughter
     // reference to a hfile link.  This code in StoreFile that handles this case.
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHdfsSnapshotHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHdfsSnapshotHRegion.java
index 6c20b5b..205f4cb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHdfsSnapshotHRegion.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHdfsSnapshotHRegion.java
@@ -29,7 +29,7 @@ import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.junit.After;
 import org.junit.Assert;
@@ -81,7 +81,7 @@ public class TestHdfsSnapshotHRegion {
     String snapshotDir = client.createSnapshot(baseDir, SNAPSHOT_NAME);
     RegionInfo firstRegion = TEST_UTIL.getConnection().getRegionLocator(
         table.getName()).getAllRegionLocations().stream().findFirst().get().getRegion();
-    Path tableDir = FSUtils.getTableDir(new Path(snapshotDir), TABLE_NAME);
+    Path tableDir = CommonFSUtils.getTableDir(new Path(snapshotDir), TABLE_NAME);
     HRegion snapshottedRegion = openSnapshotRegion(firstRegion, tableDir);
     Assert.assertNotNull(snapshottedRegion);
     snapshottedRegion.close();
@@ -93,7 +93,7 @@ public class TestHdfsSnapshotHRegion {
     RegionInfo firstRegion = TEST_UTIL.getConnection().getRegionLocator(
         table.getName()).getAllRegionLocations().stream().findFirst().get().getRegion();
     String encodedName = firstRegion.getEncodedName();
-    Path tableDir = FSUtils.getTableDir(TEST_UTIL.getDefaultRootDirPath(), TABLE_NAME);
+    Path tableDir = CommonFSUtils.getTableDir(TEST_UTIL.getDefaultRootDirPath(), TABLE_NAME);
     Path regionDirectoryPath = new Path(tableDir, encodedName);
     TEST_UTIL.getTestFileSystem().create(
         new Path(regionDirectoryPath, HRegionFileSystem.REGION_TEMP_DIR));
@@ -105,7 +105,7 @@ public class TestHdfsSnapshotHRegion {
     String snapshotDir = client.createSnapshot(baseDir, "foo_snapshot");
     // everything should still open just fine
     HRegion snapshottedRegion = openSnapshotRegion(firstRegion,
-        FSUtils.getTableDir(new Path(snapshotDir), TABLE_NAME));
+      CommonFSUtils.getTableDir(new Path(snapshotDir), TABLE_NAME));
     Assert.assertNotNull(snapshottedRegion); // no errors and the region should open
     snapshottedRegion.close();
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreCompaction.java
index 74c8d79..25e1223 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreCompaction.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreCompaction.java
@@ -31,7 +31,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Random;
 import java.util.Set;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -65,7 +64,7 @@ import org.apache.hadoop.hbase.mob.MobFileCache;
 import org.apache.hadoop.hbase.mob.MobUtils;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.Pair;
 import org.junit.After;
 import org.junit.ClassRule;
@@ -211,7 +210,7 @@ public class TestMobStoreCompaction {
     init(UTIL.getConfiguration(), 300);
     byte[] dummyData = makeDummyData(600);
 
-    Path hbaseRootDir = FSUtils.getRootDir(conf);
+    Path hbaseRootDir = CommonFSUtils.getRootDir(conf);
     Path basedir = new Path(hbaseRootDir, htd.getNameAsString());
     List<Pair<byte[], String>> hfiles = new ArrayList<>(1);
     for (int i = 0; i < compactionThreshold; i++) {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreScanner.java
index 1a73cf4..aaea39f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreScanner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreScanner.java
@@ -48,7 +48,7 @@ import org.apache.hadoop.hbase.mob.MobTestUtil;
 import org.apache.hadoop.hbase.mob.MobUtils;
 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.CommonFSUtils;
 import org.apache.hadoop.hbase.util.HFileArchiveUtil;
 import org.junit.AfterClass;
 import org.junit.Assert;
@@ -408,8 +408,8 @@ public class TestMobStoreScanner {
     FileStatus[] files = fs.listStatus(mobFamilyPath);
 
     // Get the archive path
-    Path rootDir = FSUtils.getRootDir(TEST_UTIL.getConfiguration());
-    Path tableDir = FSUtils.getTableDir(rootDir, tn);
+    Path rootDir = CommonFSUtils.getRootDir(TEST_UTIL.getConfiguration());
+    Path tableDir = CommonFSUtils.getTableDir(rootDir, tn);
     RegionInfo regionInfo = MobUtils.getMobRegionInfo(tn);
     Path storeArchiveDir = HFileArchiveUtil.getStoreArchivePath(TEST_UTIL.getConfiguration(),
         regionInfo, tableDir, family);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRecoveredEdits.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRecoveredEdits.java
index 10b9d8f..fadb69a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRecoveredEdits.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRecoveredEdits.java
@@ -24,7 +24,6 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -46,6 +45,7 @@ import org.apache.hadoop.hbase.io.hfile.BlockCache;
 import org.apache.hadoop.hbase.io.hfile.BlockCacheFactory;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.wal.WALEdit;
@@ -133,7 +133,7 @@ public class TestRecoveredEdits {
     };
     Path hbaseRootDir = TEST_UTIL.getDataTestDir();
     FileSystem fs = FileSystem.get(TEST_UTIL.getConfiguration());
-    Path tableDir = FSUtils.getTableDir(hbaseRootDir, tableDescriptor.getTableName());
+    Path tableDir = CommonFSUtils.getTableDir(hbaseRootDir, tableDescriptor.getTableName());
     HRegionFileSystem hrfs =
         new HRegionFileSystem(TEST_UTIL.getConfiguration(), fs, tableDir, hri);
     if (fs.exists(hrfs.getRegionDir())) {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRecoveredEditsReplayAndAbort.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRecoveredEditsReplayAndAbort.java
index d9f7766..37b6a82 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRecoveredEditsReplayAndAbort.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRecoveredEditsReplayAndAbort.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.regionserver;
 
 import java.io.IOException;
 import java.util.Random;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -40,7 +39,7 @@ import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.CancelableProgressable;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALFactory;
@@ -134,7 +133,7 @@ public class TestRecoveredEditsReplayAndAbort {
         .getDataTestDirOnTestFS("TestRecoveredEidtsReplayAndAbort.log");
     final WAL wal = HBaseTestingUtility.createWal(CONF, logDir, info);
     Path rootDir = TEST_UTIL.getDataTestDir();
-    Path tableDir = FSUtils.getTableDir(rootDir, info.getTable());
+    Path tableDir = CommonFSUtils.getTableDir(rootDir, info.getTable());
     HRegionFileSystem
         .createRegionOnFileSystem(CONF, TEST_UTIL.getTestFileSystem(), tableDir, info);
     region = HRegion.newHRegion(tableDir, wal, TEST_UTIL.getTestFileSystem(), CONF, info,
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 7f56f2d..7c1125d 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
@@ -21,6 +21,7 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
+
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -62,7 +63,7 @@ import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.FutureUtils;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
 import org.apache.hadoop.hbase.util.Pair;
@@ -79,9 +80,11 @@ import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.com.google.common.base.Joiner;
 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.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse;
@@ -230,7 +233,7 @@ public class TestRegionMergeTransactionOnCluster {
       FileSystem fs = MASTER.getMasterFileSystem().getFileSystem();
       Path rootDir = MASTER.getMasterFileSystem().getRootDir();
 
-      Path tabledir = FSUtils.getTableDir(rootDir, mergedRegionInfo.getTable());
+      Path tabledir = CommonFSUtils.getTableDir(rootDir, mergedRegionInfo.getTable());
       Path regionAdir = new Path(tabledir, regionA.getEncodedName());
       Path regionBdir = new Path(tabledir, regionB.getEncodedName());
       assertTrue(fs.exists(regionAdir));
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 bc1991f..acdb31e 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
@@ -40,7 +40,7 @@ import org.apache.hadoop.hbase.executor.ExecutorType;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
@@ -117,7 +117,7 @@ public class TestRegionOpen {
     HRegionInfo hri = new HRegionInfo(htd.getTableName(),  Bytes.toBytes("A"), Bytes.toBytes("B"), false,
         System.currentTimeMillis(), 2);
     HRegionFileSystem regionFs = HRegionFileSystem.createRegionOnFileSystem(conf, fs,
-        FSUtils.getTableDir(rootDir, hri.getTable()), hri);
+      CommonFSUtils.getTableDir(rootDir, hri.getTable()), hri);
     Path regionDir = regionFs.getRegionDir();
     try {
       HRegionFileSystem.loadRegionInfoFileContent(fs, regionDir);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowPrefixBloomFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowPrefixBloomFilter.java
index c0e47a0..166783f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowPrefixBloomFilter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowPrefixBloomFilter.java
@@ -25,7 +25,6 @@ import static org.mockito.Mockito.when;
 
 import java.io.IOException;
 import java.util.concurrent.atomic.AtomicInteger;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -48,7 +47,7 @@ import org.apache.hadoop.hbase.util.BloomFilterFactory;
 import org.apache.hadoop.hbase.util.BloomFilterUtil;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ChecksumType;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.ClassRule;
@@ -112,7 +111,7 @@ public class TestRowPrefixBloomFilter {
           fs.delete(testDir, true);
         }
       } else {
-        testDir = FSUtils.getRootDir(conf);
+        testDir = CommonFSUtils.getRootDir(conf);
       }
     } catch (Exception e) {
       LOG.error(HBaseMarkers.FATAL, "error during setup", e);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
index 5d63c76..45f35ef 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
@@ -90,6 +90,7 @@ import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.HBaseFsck;
@@ -677,7 +678,7 @@ public class TestSplitTransactionOnCluster {
       printOutRegions(regionServer, "Initial regions: ");
       Configuration conf = cluster.getConfiguration();
       HBaseFsck.debugLsr(conf, new Path("/"));
-      Path rootDir = FSUtils.getRootDir(conf);
+      Path rootDir = CommonFSUtils.getRootDir(conf);
       FileSystem fs = TESTING_UTIL.getDFSCluster().getFileSystem();
       Map<String, Path> storefiles =
           FSUtils.getTableStoreFilePathMap(null, fs, rootDir, tableName);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java
index e2f7a8b..3d24dd1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java
@@ -22,6 +22,7 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
+
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -49,7 +50,7 @@ import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.StoppableImplementation;
 import org.apache.hadoop.hbase.wal.WALFactory;
 import org.junit.Before;
@@ -76,7 +77,7 @@ public class TestStoreFileRefresherChore {
   public void setUp() throws IOException {
     TEST_UTIL = new HBaseTestingUtility();
     testDir = TEST_UTIL.getDataTestDir("TestStoreFileRefresherChore");
-    FSUtils.setRootDir(TEST_UTIL.getConfiguration(), testDir);
+    CommonFSUtils.setRootDir(TEST_UTIL.getConfiguration(), testDir);
   }
 
   private TableDescriptor getTableDesc(TableName tableName, int regionReplication,
@@ -108,14 +109,14 @@ public class TestStoreFileRefresherChore {
   private HRegion initHRegion(TableDescriptor htd, byte[] startKey, byte[] stopKey, int replicaId)
       throws IOException {
     Configuration conf = TEST_UTIL.getConfiguration();
-    Path tableDir = FSUtils.getTableDir(testDir, htd.getTableName());
+    Path tableDir = CommonFSUtils.getTableDir(testDir, htd.getTableName());
 
     RegionInfo info = RegionInfoBuilder.newBuilder(htd.getTableName()).setStartKey(startKey)
         .setEndKey(stopKey).setRegionId(0L).setReplicaId(replicaId).build();
     HRegionFileSystem fs =
         new FailingHRegionFileSystem(conf, tableDir.getFileSystem(conf), tableDir, info);
     final Configuration walConf = new Configuration(conf);
-    FSUtils.setRootDir(walConf, tableDir);
+    CommonFSUtils.setRootDir(walConf, tableDir);
     final WALFactory wals = new WALFactory(walConf, "log_" + replicaId);
     ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     HRegion region =
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java
index 599260b..35f4c6e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java
@@ -43,7 +43,7 @@ import org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALFactory;
@@ -113,13 +113,13 @@ public class TestWALMonotonicallyIncreasingSeqId {
     Configuration conf = TEST_UTIL.getConfiguration();
     conf.set("hbase.wal.provider", walProvider);
     conf.setBoolean("hbase.hregion.mvcc.preassign", false);
-    Path tableDir = FSUtils.getTableDir(testDir, htd.getTableName());
+    Path tableDir = CommonFSUtils.getTableDir(testDir, htd.getTableName());
 
     RegionInfo info = RegionInfoBuilder.newBuilder(htd.getTableName()).setStartKey(startKey)
         .setEndKey(stopKey).setReplicaId(replicaId).setRegionId(0).build();
     fileSystem = tableDir.getFileSystem(conf);
     final Configuration walConf = new Configuration(conf);
-    FSUtils.setRootDir(walConf, tableDir);
+    CommonFSUtils.setRootDir(walConf, tableDir);
     this.walConf = walConf;
     wals = new WALFactory(walConf, "log_" + replicaId);
     ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestFSWAL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestFSWAL.java
index 513a15f..2dd5f87 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestFSWAL.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestFSWAL.java
@@ -74,7 +74,6 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.EnvironmentEdge;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
 import org.apache.hadoop.hbase.wal.WAL;
@@ -499,7 +498,7 @@ public abstract class AbstractTestFSWAL {
     final CountDownLatch closeFinished = new CountDownLatch(1);
     final CountDownLatch putFinished = new CountDownLatch(1);
 
-    try (AbstractFSWAL<?> wal = newWAL(FS, FSUtils.getRootDir(CONF), testName,
+    try (AbstractFSWAL<?> wal = newWAL(FS, CommonFSUtils.getRootDir(CONF), testName,
       HConstants.HREGION_OLDLOGDIR_NAME, CONF, null, true, null, null)) {
       wal.init();
       wal.registerWALActionsListener(new WALActionsListener() {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java
index 98d2db1..dbbe228 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java
@@ -85,10 +85,10 @@ import org.apache.hadoop.hbase.regionserver.RegionServerServices;
 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.CommonFSUtils.StreamLacksCapabilityException;
 import org.apache.hadoop.hbase.util.EnvironmentEdge;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.HFileTestUtil;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
@@ -140,7 +140,7 @@ public abstract class AbstractTestWALReplay {
     Path hbaseRootDir =
       TEST_UTIL.getDFSCluster().getFileSystem().makeQualified(new Path("/hbase"));
     LOG.info("hbase.rootdir=" + hbaseRootDir);
-    FSUtils.setRootDir(conf, hbaseRootDir);
+    CommonFSUtils.setRootDir(conf, hbaseRootDir);
   }
 
   @AfterClass
@@ -152,7 +152,7 @@ public abstract class AbstractTestWALReplay {
   public void setUp() throws Exception {
     this.conf = HBaseConfiguration.create(TEST_UTIL.getConfiguration());
     this.fs = TEST_UTIL.getDFSCluster().getFileSystem();
-    this.hbaseRootDir = FSUtils.getRootDir(this.conf);
+    this.hbaseRootDir = CommonFSUtils.getRootDir(this.conf);
     this.oldLogDir = new Path(this.hbaseRootDir, HConstants.HREGION_OLDLOGDIR_NAME);
     String serverName =
       ServerName.valueOf(currentTest.getMethodName() + "-manual", 16010, System.currentTimeMillis())
@@ -275,7 +275,7 @@ public abstract class AbstractTestWALReplay {
 
     MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl();
     HRegionInfo hri = createBasic3FamilyHRegionInfo(tableName);
-    Path basedir = FSUtils.getTableDir(hbaseRootDir, tableName);
+    Path basedir = CommonFSUtils.getTableDir(hbaseRootDir, tableName);
     deleteDir(basedir);
 
     HTableDescriptor htd = createBasic3FamilyHTD(tableName);
@@ -477,7 +477,7 @@ public abstract class AbstractTestWALReplay {
     final TableName tableName =
         TableName.valueOf("testReplayEditsWrittenViaHRegion");
     final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableName);
-    final Path basedir = FSUtils.getTableDir(this.hbaseRootDir, tableName);
+    final Path basedir = CommonFSUtils.getTableDir(this.hbaseRootDir, tableName);
     deleteDir(basedir);
     final byte[] rowName = tableName.getName();
     final int countPerFamily = 10;
@@ -585,7 +585,7 @@ public abstract class AbstractTestWALReplay {
     final TableName tableName =
         TableName.valueOf("testReplayEditsWrittenViaHRegion");
     final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableName);
-    final Path basedir = FSUtils.getTableDir(this.hbaseRootDir, tableName);
+    final Path basedir = CommonFSUtils.getTableDir(this.hbaseRootDir, tableName);
     deleteDir(basedir);
     final byte[] rowName = tableName.getName();
     final int countPerFamily = 10;
@@ -670,7 +670,7 @@ public abstract class AbstractTestWALReplay {
     final TableName tableName =
         TableName.valueOf("testReplayEditsAfterAbortingFlush");
     final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableName);
-    final Path basedir = FSUtils.getTableDir(this.hbaseRootDir, tableName);
+    final Path basedir = CommonFSUtils.getTableDir(this.hbaseRootDir, tableName);
     deleteDir(basedir);
     final HTableDescriptor htd = createBasic3FamilyHTD(tableName);
     HRegion region3 = HBaseTestingUtility.createRegionAndWAL(hri, hbaseRootDir, this.conf, htd);
@@ -769,7 +769,7 @@ public abstract class AbstractTestWALReplay {
         TableName.valueOf("testReplayEditsWrittenIntoWAL");
     final MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl();
     final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableName);
-    final Path basedir = FSUtils.getTableDir(hbaseRootDir, tableName);
+    final Path basedir = CommonFSUtils.getTableDir(hbaseRootDir, tableName);
     deleteDir(basedir);
 
     final HTableDescriptor htd = createBasic3FamilyHTD(tableName);
@@ -871,7 +871,7 @@ public abstract class AbstractTestWALReplay {
     final TableName tableName = TableName.valueOf(currentTest.getMethodName());
     final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableName);
     final Path basedir =
-        FSUtils.getWALTableDir(conf, tableName);
+      CommonFSUtils.getWALTableDir(conf, tableName);
     deleteDir(basedir);
     final byte[] rowName = tableName.getName();
     final int countPerFamily = 10;
@@ -903,14 +903,14 @@ public abstract class AbstractTestWALReplay {
     assertTrue(listStatus.length > 0);
     WALSplitter.splitLogFile(hbaseRootDir, listStatus[0], this.fs, this.conf, null, null, null,
       wals, null);
-    FileStatus[] listStatus1 = this.fs.listStatus(new Path(FSUtils.getWALTableDir(conf, tableName),
-        new Path(hri.getEncodedName(), "recovered.edits")),
-      new PathFilter() {
-        @Override
-        public boolean accept(Path p) {
-          return !WALSplitUtil.isSequenceIdFile(p);
-        }
-      });
+    FileStatus[] listStatus1 =
+      this.fs.listStatus(new Path(CommonFSUtils.getWALTableDir(conf, tableName),
+        new Path(hri.getEncodedName(), "recovered.edits")), new PathFilter() {
+          @Override
+          public boolean accept(Path p) {
+            return !WALSplitUtil.isSequenceIdFile(p);
+          }
+        });
     int editCount = 0;
     for (FileStatus fileStatus : listStatus1) {
       editCount = Integer.parseInt(fileStatus.getPath().getName());
@@ -928,7 +928,7 @@ public abstract class AbstractTestWALReplay {
   public void testDatalossWhenInputError() throws Exception {
     final TableName tableName = TableName.valueOf("testDatalossWhenInputError");
     final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableName);
-    final Path basedir = FSUtils.getWALTableDir(conf, tableName);
+    final Path basedir = CommonFSUtils.getWALTableDir(conf, tableName);
     deleteDir(basedir);
     final byte[] rowName = tableName.getName();
     final int countPerFamily = 10;
@@ -1025,7 +1025,7 @@ public abstract class AbstractTestWALReplay {
     final TableName tableName = TableName.valueOf("testReplayEditsWrittenIntoWAL");
     final MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl();
     final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableName);
-    final Path basedir = FSUtils.getTableDir(hbaseRootDir, tableName);
+    final Path basedir = CommonFSUtils.getTableDir(hbaseRootDir, tableName);
     deleteDir(basedir);
 
     final HTableDescriptor htd = createBasic1FamilyHTD(tableName);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java
index 320bcbf..2d250c8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java
@@ -44,7 +44,7 @@ import org.apache.hadoop.hbase.regionserver.MemStoreLABImpl;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
 import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.wal.WALFactory;
@@ -103,7 +103,7 @@ public class TestDurability {
     CLUSTER = TEST_UTIL.getDFSCluster();
     FS = CLUSTER.getFileSystem();
     DIR = TEST_UTIL.getDataTestDirOnTestFS("TestDurability");
-    FSUtils.setRootDir(CONF, DIR);
+    CommonFSUtils.setRootDir(CONF, DIR);
   }
 
   @AfterClass
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java
index 8f6e518..5da1bfb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java
@@ -47,7 +47,7 @@ import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALKey;
@@ -102,8 +102,8 @@ public class TestFSHLog extends AbstractTestFSWAL {
   public void testSyncRunnerIndexOverflow() throws IOException, NoSuchFieldException,
       SecurityException, IllegalArgumentException, IllegalAccessException {
     final String name = this.name.getMethodName();
-    FSHLog log = new FSHLog(FS, FSUtils.getRootDir(CONF), name, HConstants.HREGION_OLDLOGDIR_NAME,
-      CONF, null, true, null, null);
+    FSHLog log = new FSHLog(FS, CommonFSUtils.getRootDir(CONF), name,
+      HConstants.HREGION_OLDLOGDIR_NAME, CONF, null, true, null, null);
     log.init();
     try {
       Field ringBufferEventHandlerField = FSHLog.class.getDeclaredField("ringBufferEventHandler");
@@ -144,9 +144,8 @@ public class TestFSHLog extends AbstractTestFSWAL {
     final CountDownLatch flushFinished = new CountDownLatch(1);
     final CountDownLatch putFinished = new CountDownLatch(1);
 
-    try (FSHLog log =
-        new FSHLog(FS, FSUtils.getRootDir(CONF), name, HConstants.HREGION_OLDLOGDIR_NAME, CONF,
-            null, true, null, null)) {
+    try (FSHLog log = new FSHLog(FS, CommonFSUtils.getRootDir(CONF), name,
+      HConstants.HREGION_OLDLOGDIR_NAME, CONF, null, true, null, null)) {
       log.init();
       log.registerWALActionsListener(new WALActionsListener() {
         @Override
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java
index a7d4a55..d0274de 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java
@@ -45,7 +45,7 @@ import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
 import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.wal.WALEdit;
@@ -122,8 +122,8 @@ public class TestLogRollAbort {
 
     // disable region rebalancing (interferes with log watching)
     cluster.getMaster().balanceSwitch(false);
-    FSUtils.setRootDir(conf, HBASEDIR);
-    FSUtils.setWALRootDir(conf, HBASELOGDIR);
+    CommonFSUtils.setRootDir(conf, HBASEDIR);
+    CommonFSUtils.setWALRootDir(conf, HBASELOGDIR);
   }
 
   @After
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java
index 6c257e2..fb07d2a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java
@@ -51,6 +51,7 @@ import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.VerySlowRegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.JVMClusterUtil;
 import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
@@ -494,13 +495,12 @@ public class TestLogRolling extends AbstractTestLogRolling {
 
       // read back the data written
       Set<String> loggedRows = new HashSet<>();
-      FSUtils fsUtils = FSUtils.getInstance(fs, TEST_UTIL.getConfiguration());
       for (Path p : paths) {
         LOG.debug("recovering lease for " + p);
-        fsUtils.recoverFileLease(((HFileSystem) fs).getBackingFs(), p, TEST_UTIL.getConfiguration(),
+        FSUtils.recoverFileLease(((HFileSystem) fs).getBackingFs(), p, TEST_UTIL.getConfiguration(),
           null);
 
-        LOG.debug("Reading WAL " + FSUtils.getPath(p));
+        LOG.debug("Reading WAL " + CommonFSUtils.getPath(p));
         WAL.Reader reader = null;
         try {
           reader = WALFactory.createReader(fs, p, TEST_UTIL.getConfiguration());
@@ -513,7 +513,7 @@ public class TestLogRolling extends AbstractTestLogRolling {
             }
           }
         } catch (EOFException e) {
-          LOG.debug("EOF reading file " + FSUtils.getPath(p));
+          LOG.debug("EOF reading file " + CommonFSUtils.getPath(p));
         } finally {
           if (reader != null) reader.close();
         }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java
index 9ba10b9..a8b6490 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java
@@ -37,8 +37,8 @@ import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 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.util.Threads;
 import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.wal.WALEdit;
@@ -90,7 +90,7 @@ public class TestLogRollingNoCluster {
     TEST_UTIL.getConfiguration().setInt(HConstants.REGION_SERVER_HANDLER_COUNT, NUM_THREADS);
     final Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
     conf.set(WALFactory.WAL_PROVIDER, "filesystem");
-    FSUtils.setRootDir(conf, dir);
+    CommonFSUtils.setRootDir(conf, dir);
     FSTableDescriptors fsTableDescriptors = new FSTableDescriptors(TEST_UTIL.getConfiguration());
     FSTableDescriptors.tryUpdateMetaTableDescriptor(TEST_UTIL.getConfiguration());
     TableDescriptor metaTableDescriptor = fsTableDescriptors.get(TableName.META_TABLE_NAME);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java
index dd83c7c..e40d7e1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java
@@ -35,7 +35,7 @@ import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALFactory;
@@ -73,8 +73,8 @@ public class TestWALActionsListener {
     conf.setInt("hbase.regionserver.maxlogs", 5);
     rootDir = TEST_UTIL.createRootDir();
     walRootDir = TEST_UTIL.createWALRootDir();
-    fs = FSUtils.getRootDirFileSystem(conf);
-    logFs = FSUtils.getWALFileSystem(conf);
+    fs = CommonFSUtils.getRootDirFileSystem(conf);
+    logFs = CommonFSUtils.getWALFileSystem(conf);
   }
 
   @Before
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java
index aa6c39c..9145841 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java
@@ -58,7 +58,7 @@ import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.HFileTestUtil;
 import org.junit.AfterClass;
 import org.junit.Assert;
@@ -133,7 +133,7 @@ public class TestReplicationSink {
       new ReplicationSink(new Configuration(TEST_UTIL.getConfiguration()), STOPPABLE);
     table1 = TEST_UTIL.createTable(TABLE_NAME1, FAM_NAME1);
     table2 = TEST_UTIL.createTable(TABLE_NAME2, FAM_NAME2);
-    Path rootDir = FSUtils.getRootDir(TEST_UTIL.getConfiguration());
+    Path rootDir = CommonFSUtils.getRootDir(TEST_UTIL.getConfiguration());
     baseNamespaceDir = new Path(rootDir, new Path(HConstants.BASE_NAMESPACE_DIR)).toString();
     hfileArchiveDir = new Path(rootDir, new Path(HConstants.HFILE_ARCHIVE_DIRECTORY)).toString();
     replicationClusterId = "12345";
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
index ccf4e06..1e40457 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
@@ -76,7 +76,7 @@ import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.JVMClusterUtil;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.wal.WAL;
@@ -175,7 +175,7 @@ public abstract class TestReplicationSourceManager {
     ZKUtil.setData(zkw, "/hbase/replication/state", ZKReplicationPeerStorage.ENABLED_ZNODE_BYTES);
 
     ZKClusterId.setClusterId(zkw, new ClusterId());
-    FSUtils.setRootDir(utility.getConfiguration(), utility.getDataTestDir());
+    CommonFSUtils.setRootDir(utility.getConfiguration(), utility.getDataTestDir());
     fs = FileSystem.get(conf);
     oldLogDir = new Path(utility.getDataTestDir(),
         HConstants.HREGION_OLDLOGDIR_NAME);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/provider/TestCustomSaslAuthenticationProvider.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/provider/TestCustomSaslAuthenticationProvider.java
index c864895..b8c7072 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/provider/TestCustomSaslAuthenticationProvider.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/provider/TestCustomSaslAuthenticationProvider.java
@@ -22,6 +22,7 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
+
 import java.io.ByteArrayInputStream;
 import java.io.DataInput;
 import java.io.DataInputStream;
@@ -85,7 +86,7 @@ import org.apache.hadoop.hbase.security.token.TokenProvider;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.SecurityTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.WritableUtils;
@@ -108,7 +109,9 @@ import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.com.google.common.base.Throwables;
+
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation;
 
 /**
@@ -442,7 +445,7 @@ public class TestCustomSaslAuthenticationProvider {
         TokenProvider.class.getName());
     util.startMiniDFSCluster(1);
     Path rootdir = util.getDataTestDirOnTestFS("TestGenerateDelegationToken");
-    FSUtils.setRootDir(util.getConfiguration(), rootdir);
+    CommonFSUtils.setRootDir(util.getConfiguration(), rootdir);
   }
 
   private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/SecureTestCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/SecureTestCluster.java
index 1323946..bff7d9d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/SecureTestCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/SecureTestCluster.java
@@ -18,19 +18,18 @@
 
 package org.apache.hadoop.hbase.security.token;
 
+import java.io.File;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.LocalHBaseCluster;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.security.HBaseKerberosUtils;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.minikdc.MiniKdc;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 
-import java.io.File;
-
 /**
  * The class for set up a security cluster with kerberos, hdfs, hbase.
  */
@@ -88,7 +87,7 @@ public class SecureTestCluster {
         TokenProvider.class.getName());
     TEST_UTIL.startMiniDFSCluster(1);
     Path rootdir = TEST_UTIL.getDataTestDirOnTestFS("TestGenerateDelegationToken");
-    FSUtils.setRootDir(TEST_UTIL.getConfiguration(), rootdir);
+    CommonFSUtils.setRootDir(TEST_UTIL.getConfiguration(), rootdir);
     CLUSTER = new LocalHBaseCluster(TEST_UTIL.getConfiguration(), 1);
     CLUSTER.startup();
   }
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 d019795..42b21e9 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
@@ -63,9 +63,9 @@ import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSTableDescriptors;
-import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.FSVisitor;
 import org.apache.hadoop.hbase.util.MD5Hash;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -583,7 +583,7 @@ public final class SnapshotTestingUtils {
        * @throws IOException on unexecpted error from the FS
        */
       public void corruptOneRegionManifest() throws IOException {
-        FileStatus[] manifestFiles = FSUtils.listStatus(fs, snapshotDir, new PathFilter() {
+        FileStatus[] manifestFiles = CommonFSUtils.listStatus(fs, snapshotDir, new PathFilter() {
           @Override public boolean accept(Path path) {
             return path.getName().startsWith(SnapshotManifestV2.SNAPSHOT_MANIFEST_PREFIX);
           }
@@ -597,7 +597,7 @@ public final class SnapshotTestingUtils {
       }
 
       public void missOneRegionSnapshotFile() throws IOException {
-        FileStatus[] manifestFiles = FSUtils.listStatus(fs, snapshotDir);
+        FileStatus[] manifestFiles = CommonFSUtils.listStatus(fs, snapshotDir);
         for (FileStatus fileStatus : manifestFiles) {
           String fileName = fileStatus.getPath().getName();
           if (fileName.endsWith(SnapshotDescriptionUtils.SNAPSHOTINFO_FILE)
@@ -614,7 +614,7 @@ public final class SnapshotTestingUtils {
        * @throws IOException on unexecpted error from the FS
        */
       public void corruptDataManifest() throws IOException {
-        FileStatus[] manifestFiles = FSUtils.listStatus(fs, snapshotDir, new PathFilter() {
+        FileStatus[] manifestFiles = CommonFSUtils.listStatus(fs, snapshotDir, new PathFilter() {
           @Override
           public boolean accept(Path path) {
             return path.getName().startsWith(SnapshotManifest.DATA_MANIFEST_NAME);
@@ -723,7 +723,7 @@ public final class SnapshotTestingUtils {
 
     private RegionData[] createTable(final TableDescriptor htd, final int nregions)
         throws IOException {
-      Path tableDir = FSUtils.getTableDir(rootDir, htd.getTableName());
+      Path tableDir = CommonFSUtils.getTableDir(rootDir, htd.getTableName());
       new FSTableDescriptors(conf).createTableDescriptorForTableDirectory(tableDir, htd, false);
 
       assertTrue(nregions % 2 == 0);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRegionSnapshotTask.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRegionSnapshotTask.java
index 17674af..11a73cb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRegionSnapshotTask.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRegionSnapshotTask.java
@@ -45,7 +45,7 @@ import org.apache.hadoop.hbase.regionserver.snapshot.FlushSnapshotSubprocedure;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
@@ -88,7 +88,7 @@ public class TestRegionSnapshotTask {
     TEST_UTIL.getHBaseCluster().waitForActiveAndReadyMaster();
     TEST_UTIL.waitUntilAllRegionsAssigned(TableName.META_TABLE_NAME);
 
-    rootDir = FSUtils.getRootDir(conf);
+    rootDir = CommonFSUtils.getRootDir(conf);
     fs = TEST_UTIL.getTestFileSystem();
   }
 
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 3a21405..14e290c 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
@@ -46,7 +46,6 @@ import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 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.WALSplitUtil;
 import org.junit.After;
 import org.junit.AfterClass;
@@ -102,7 +101,7 @@ public class TestRestoreSnapshotHelper {
     fs = TEST_UTIL.getTestFileSystem();
     conf = TEST_UTIL.getConfiguration();
     setupConf(conf);
-    FSUtils.setRootDir(conf, rootDir);
+    CommonFSUtils.setRootDir(conf, rootDir);
   }
 
   @After
@@ -127,7 +126,7 @@ public class TestRestoreSnapshotHelper {
   @Test
   public void testNoHFileLinkInRootDir() throws IOException {
     rootDir = TEST_UTIL.getDefaultRootDirPath();
-    FSUtils.setRootDir(conf, rootDir);
+    CommonFSUtils.setRootDir(conf, rootDir);
     fs = rootDir.getFileSystem(conf);
 
     TableName tableName = TableName.valueOf("testNoHFileLinkInRootDir");
@@ -142,7 +141,7 @@ public class TestRestoreSnapshotHelper {
   @Test
   public void testSkipReplayAndUpdateSeqId() throws Exception {
     rootDir = TEST_UTIL.getDefaultRootDirPath();
-    FSUtils.setRootDir(conf, rootDir);
+    CommonFSUtils.setRootDir(conf, rootDir);
     TableName tableName = TableName.valueOf("testSkipReplayAndUpdateSeqId");
     String snapshotName = "testSkipReplayAndUpdateSeqId";
     createTableAndSnapshot(tableName, snapshotName);
@@ -151,27 +150,27 @@ public class TestRestoreSnapshotHelper {
     TEST_UTIL.loadTable(table, Bytes.toBytes("A"));
 
     Configuration conf = TEST_UTIL.getConfiguration();
-    Path rootDir = FSUtils.getRootDir(conf);
+    Path rootDir = CommonFSUtils.getRootDir(conf);
     Path restoreDir = new Path("/hbase/.tmp-restore/testScannerWithRestoreScanner2");
     // restore snapshot.
     final RestoreSnapshotHelper.RestoreMetaChanges meta =
-        RestoreSnapshotHelper.copySnapshotForScanner(conf, fs, rootDir, restoreDir, snapshotName);
+      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);
+      HRegion region = HRegion.newHRegion(CommonFSUtils.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());
+        CommonFSUtils.getWALRegionDir(conf, tableName, region.getRegionInfo().getEncodedName());
       long maxSeqId = WALSplitUtil.getMaxRegionSequenceId(fs, recoveredEdit);
 
       // open restored region without set restored flag
-      HRegion region2 = HRegion.newHRegion(FSUtils.getTableDir(restoreDir, tableName), null, fs,
-        conf, restoredRegion, htd, null);
+      HRegion region2 = HRegion.newHRegion(CommonFSUtils.getTableDir(restoreDir, tableName), null,
+        fs, conf, restoredRegion, htd, null);
       region2.initialize();
       long maxSeqId2 = WALSplitUtil.getMaxRegionSequenceId(fs, recoveredEdit);
       Assert.assertTrue(maxSeqId2 > maxSeqId);
@@ -231,7 +230,7 @@ public class TestRestoreSnapshotHelper {
         .setName("cloneSnapshot")
         .setTable("testtb-clone")
         .build();
-    Path cloneDir = FSUtils.getTableDir(rootDir, htdClone.getTableName());
+    Path cloneDir = CommonFSUtils.getTableDir(rootDir, htdClone.getTableName());
     TableDescriptor htdClone2 = snapshotMock.createHtd("testtb-clone2");
     testRestore(cloneDir, cloneDesc, htdClone2);
     verifyRestore(rootDir, htd, htdClone2);
@@ -240,7 +239,7 @@ public class TestRestoreSnapshotHelper {
   private void verifyRestore(final Path rootDir, final TableDescriptor sourceHtd,
       final TableDescriptor htdClone) throws IOException {
     List<String> files = SnapshotTestingUtils.listHFileNames(fs,
-      FSUtils.getTableDir(rootDir, htdClone.getTableName()));
+      CommonFSUtils.getTableDir(rootDir, htdClone.getTableName()));
     assertEquals(12, files.size());
     for (int i = 0; i < files.size(); i += 2) {
       String linkFile = files.get(i);
@@ -264,14 +263,14 @@ public class TestRestoreSnapshotHelper {
   private void testRestore(final Path snapshotDir, final SnapshotDescription sd,
       final TableDescriptor htdClone) throws IOException {
     LOG.debug("pre-restore table=" + htdClone.getTableName() + " snapshot=" + snapshotDir);
-    FSUtils.logFileSystemState(fs, rootDir, LOG);
+    CommonFSUtils.logFileSystemState(fs, rootDir, LOG);
 
     new FSTableDescriptors(conf).createTableDescriptor(htdClone);
     RestoreSnapshotHelper helper = getRestoreHelper(rootDir, snapshotDir, sd, htdClone);
     helper.restoreHdfsRegions();
 
     LOG.debug("post-restore table=" + htdClone.getTableName() + " snapshot=" + snapshotDir);
-    FSUtils.logFileSystemState(fs, rootDir, LOG);
+    CommonFSUtils.logFileSystemState(fs, rootDir, LOG);
   }
 
   /**
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotStoreFileSize.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotStoreFileSize.java
index 9baa4e2..f49ff4a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotStoreFileSize.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotStoreFileSize.java
@@ -36,13 +36,14 @@ import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
 
@@ -105,7 +106,7 @@ public class TestSnapshotStoreFileSize {
       }
     }
     List<RegionInfo> regionsInfo = admin.getRegions(TABLE_NAME);
-    Path path = FSUtils.getTableDir(UTIL.getDefaultRootDirPath(), TABLE_NAME);
+    Path path = CommonFSUtils.getTableDir(UTIL.getDefaultRootDirPath(), TABLE_NAME);
     for (RegionInfo regionInfo : regionsInfo) {
       HRegionFileSystem hRegionFileSystem =
           HRegionFileSystem.openRegionFromFileSystem(conf, fs, path, regionInfo, true);
@@ -113,7 +114,7 @@ public class TestSnapshotStoreFileSize {
       Iterator<StoreFileInfo> sfIterator = storeFilesFS.iterator();
       while (sfIterator.hasNext()) {
         StoreFileInfo sfi = sfIterator.next();
-        FileStatus[] fileStatus = FSUtils.listStatus(fs, sfi.getPath());
+        FileStatus[] fileStatus = CommonFSUtils.listStatus(fs, sfi.getPath());
         storeFileName = fileStatus[0].getPath().getName();
         storeFilesize = fileStatus[0].getLen();
         storeFileInfoFromFS.put(storeFileName, storeFilesize);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotWhenChoreCleaning.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotWhenChoreCleaning.java
index 182e3b0..ac48940 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotWhenChoreCleaning.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotWhenChoreCleaning.java
@@ -23,7 +23,6 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicBoolean;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -39,7 +38,7 @@ import org.apache.hadoop.hbase.master.snapshot.SnapshotHFileCleaner;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 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.CommonFSUtils;
 import org.apache.hadoop.hbase.util.FSVisitor;
 import org.junit.AfterClass;
 import org.junit.Assert;
@@ -126,7 +125,7 @@ public class TestSnapshotWhenChoreCleaning {
   }
 
   private static boolean isAnySnapshots(FileSystem fs) throws IOException {
-    Path snapshotDir = SnapshotDescriptionUtils.getSnapshotsDir(FSUtils.getRootDir(CONF));
+    Path snapshotDir = SnapshotDescriptionUtils.getSnapshotsDir(CommonFSUtils.getRootDir(CONF));
     FileStatus[] snapFiles = fs.listStatus(snapshotDir);
     if (snapFiles.length == 0) {
       return false;
@@ -149,9 +148,9 @@ public class TestSnapshotWhenChoreCleaning {
     cleaner.init(ImmutableMap.of(HMaster.MASTER, TEST_UTIL.getHBaseCluster().getMaster()));
     cleaner.setConf(CONF);
 
-    FileSystem fs = FSUtils.getCurrentFileSystem(CONF);
+    FileSystem fs = CommonFSUtils.getCurrentFileSystem(CONF);
     List<Path> fileNames =
-        listHFileNames(fs, FSUtils.getTableDir(FSUtils.getRootDir(CONF), TABLE_NAME));
+        listHFileNames(fs, CommonFSUtils.getTableDir(CommonFSUtils.getRootDir(CONF), TABLE_NAME));
     List<FileStatus> files = new ArrayList<>();
     for (Path fileName : fileNames) {
       files.add(fs.getFileStatus(fileName));
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestLoadIncrementalHFiles.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestLoadIncrementalHFiles.java
index badfc2a..8d3d04d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestLoadIncrementalHFiles.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestLoadIncrementalHFiles.java
@@ -55,7 +55,7 @@ import org.apache.hadoop.hbase.regionserver.BloomType;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.HFileTestUtil;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -392,8 +392,8 @@ public class TestLoadIncrementalHFiles {
 
     final TableName tableName = htd.getTableName();
     // verify staging folder has been cleaned up
-    Path stagingBasePath =
-        new Path(FSUtils.getRootDir(util.getConfiguration()), HConstants.BULKLOAD_STAGING_DIR_NAME);
+    Path stagingBasePath = new Path(CommonFSUtils.getRootDir(util.getConfiguration()),
+      HConstants.BULKLOAD_STAGING_DIR_NAME);
     FileSystem fs = util.getTestFileSystem();
     if (fs.exists(stagingBasePath)) {
       FileStatus[] files = fs.listStatus(stagingBasePath);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestLoadIncrementalHFilesSplitRecovery.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestLoadIncrementalHFilesSplitRecovery.java
index 17ee0f6..aae529b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestLoadIncrementalHFilesSplitRecovery.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestLoadIncrementalHFilesSplitRecovery.java
@@ -67,7 +67,7 @@ import org.apache.hadoop.hbase.regionserver.TestHRegionServerBulkLoad;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.Pair;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -504,7 +504,7 @@ public class TestLoadIncrementalHFilesSplitRecovery {
       assertTrue(fs.exists(tmpPath));
       // TMP_DIR should have been cleaned-up
       assertNull(LoadIncrementalHFiles.TMP_DIR + " should be empty.",
-        FSUtils.listStatus(fs, tmpPath));
+        CommonFSUtils.listStatus(fs, tmpPath));
       assertExpectedTable(connection, table, ROWCOUNT, 2);
     }
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java
index cc1805a..5ed5413 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java
@@ -183,9 +183,9 @@ public class BaseTestHBaseFsck {
 
         if (regionInfoOnly) {
           LOG.info("deleting hdfs .regioninfo data: " + hri.toString() + hsa.toString());
-          Path rootDir = FSUtils.getRootDir(conf);
+          Path rootDir = CommonFSUtils.getRootDir(conf);
           FileSystem fs = rootDir.getFileSystem(conf);
-          Path p = new Path(FSUtils.getTableDir(rootDir, htd.getTableName()),
+          Path p = new Path(CommonFSUtils.getTableDir(rootDir, htd.getTableName()),
               hri.getEncodedName());
           Path hriPath = new Path(p, HRegionFileSystem.REGION_INFO_FILE);
           fs.delete(hriPath, true);
@@ -193,9 +193,9 @@ public class BaseTestHBaseFsck {
 
         if (hdfs) {
           LOG.info("deleting hdfs data: " + hri.toString() + hsa.toString());
-          Path rootDir = FSUtils.getRootDir(conf);
+          Path rootDir = CommonFSUtils.getRootDir(conf);
           FileSystem fs = rootDir.getFileSystem(conf);
-          Path p = new Path(FSUtils.getTableDir(rootDir, htd.getTableName()),
+          Path p = new Path(CommonFSUtils.getTableDir(rootDir, htd.getTableName()),
               hri.getEncodedName());
           HBaseFsck.debugLsr(conf, p);
           boolean success = fs.delete(p, true);
@@ -343,9 +343,9 @@ public class BaseTestHBaseFsck {
   }
 
   public void deleteTableDir(TableName table) throws IOException {
-    Path rootDir = FSUtils.getRootDir(conf);
+    Path rootDir = CommonFSUtils.getRootDir(conf);
     FileSystem fs = rootDir.getFileSystem(conf);
-    Path p = FSUtils.getTableDir(rootDir, table);
+    Path p = CommonFSUtils.getTableDir(rootDir, table);
     HBaseFsck.debugLsr(conf, p);
     boolean success = fs.delete(p, true);
     LOG.info("Deleted " + p + " sucessfully? " + success);
@@ -360,7 +360,7 @@ public class BaseTestHBaseFsck {
    * @throws IOException
    */
   Path getFlushedHFile(FileSystem fs, TableName table) throws IOException {
-    Path tableDir= FSUtils.getTableDir(FSUtils.getRootDir(conf), table);
+    Path tableDir= CommonFSUtils.getTableDir(CommonFSUtils.getRootDir(conf), table);
     Path regionDir = FSUtils.getRegionDirs(fs, tableDir).get(0);
     Path famDir = new Path(regionDir, FAM_STR);
 
@@ -553,7 +553,7 @@ public class BaseTestHBaseFsck {
 
     if (regionInfoOnly) {
       LOG.info("deleting hdfs .regioninfo data: " + hri.toString() + hsa.toString());
-      Path rootDir = FSUtils.getRootDir(conf);
+      Path rootDir = CommonFSUtils.getRootDir(conf);
       FileSystem fs = rootDir.getFileSystem(conf);
       Path p = new Path(rootDir + "/" + TableName.META_TABLE_NAME.getNameAsString(),
           hri.getEncodedName());
@@ -563,7 +563,7 @@ public class BaseTestHBaseFsck {
 
     if (hdfs) {
       LOG.info("deleting hdfs data: " + hri.toString() + hsa.toString());
-      Path rootDir = FSUtils.getRootDir(conf);
+      Path rootDir = CommonFSUtils.getRootDir(conf);
       FileSystem fs = rootDir.getFileSystem(conf);
       Path p = new Path(rootDir + "/" + TableName.META_TABLE_NAME.getNameAsString(),
           hri.getEncodedName());
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileArchiveTestingUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileArchiveTestingUtil.java
index c58362c..1956f5a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileArchiveTestingUtil.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileArchiveTestingUtil.java
@@ -25,7 +25,6 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -211,10 +210,8 @@ public class HFileArchiveTestingUtil {
    * @return {@link Path} to the archive directory for the given region
    */
   public static Path getRegionArchiveDir(Configuration conf, HRegion region) throws IOException {
-    return HFileArchiveUtil.getRegionArchiveDir(
-        FSUtils.getRootDir(conf),
-        region.getTableDescriptor().getTableName(),
-        region.getRegionInfo().getEncodedName());
+    return HFileArchiveUtil.getRegionArchiveDir(CommonFSUtils.getRootDir(conf),
+      region.getTableDescriptor().getTableName(), region.getRegionInfo().getEncodedName());
   }
 
   /**
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSHDFSUtils.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSHDFSUtils.java
index e88dd5f..a645f94 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSHDFSUtils.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSHDFSUtils.java
@@ -29,7 +29,6 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
-import org.junit.Before;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -40,7 +39,7 @@ import org.slf4j.LoggerFactory;
 /**
  * Test our recoverLease loop against mocked up filesystem.
  */
-@Category({MiscTests.class, MediumTests.class})
+@Category({ MiscTests.class, MediumTests.class })
 public class TestFSHDFSUtils {
 
   @ClassRule
@@ -53,21 +52,16 @@ public class TestFSHDFSUtils {
     Configuration conf = HTU.getConfiguration();
     conf.setInt("hbase.lease.recovery.first.pause", 10);
     conf.setInt("hbase.lease.recovery.pause", 10);
-  };
-  private FSHDFSUtils fsHDFSUtils = new FSHDFSUtils();
-  private static Path FILE = new Path(HTU.getDataTestDir(), "file.txt");
-  long startTime = -1;
-
-  @Before
-  public void setup() {
-    this.startTime = EnvironmentEdgeManager.currentTime();
   }
 
+  private static Path FILE = new Path(HTU.getDataTestDir(), "file.txt");
+
   /**
    * Test recover lease eventually succeeding.
    */
   @Test
   public void testRecoverLease() throws IOException {
+    long startTime = EnvironmentEdgeManager.currentTime();
     HTU.getConfiguration().setInt("hbase.lease.recovery.dfs.timeout", 1000);
     CancelableProgressable reporter = Mockito.mock(CancelableProgressable.class);
     Mockito.when(reporter.progress()).thenReturn(true);
@@ -75,11 +69,11 @@ public class TestFSHDFSUtils {
     // Fail four times and pass on the fifth.
     Mockito.when(dfs.recoverLease(FILE)).
       thenReturn(false).thenReturn(false).thenReturn(false).thenReturn(false).thenReturn(true);
-    assertTrue(this.fsHDFSUtils.recoverDFSFileLease(dfs, FILE, HTU.getConfiguration(), reporter));
+    FSUtils.recoverFileLease(dfs, FILE, HTU.getConfiguration(), reporter);
     Mockito.verify(dfs, Mockito.times(5)).recoverLease(FILE);
     // Make sure we waited at least hbase.lease.recovery.dfs.timeout * 3 (the first two
     // invocations will happen pretty fast... the we fall into the longer wait loop).
-    assertTrue((EnvironmentEdgeManager.currentTime() - this.startTime) >
+    assertTrue((EnvironmentEdgeManager.currentTime() - startTime) >
       (3 * HTU.getConfiguration().getInt("hbase.lease.recovery.dfs.timeout", 61000)));
   }
 
@@ -99,35 +93,27 @@ public class TestFSHDFSUtils {
     Mockito.when(dfs.recoverLease(FILE)).
       thenReturn(false).thenReturn(false).thenReturn(true);
     Mockito.when(dfs.isFileClosed(FILE)).thenReturn(true);
-    assertTrue(this.fsHDFSUtils.recoverDFSFileLease(dfs, FILE, HTU.getConfiguration(), reporter));
+    FSUtils.recoverFileLease(dfs, FILE, HTU.getConfiguration(), reporter);
     Mockito.verify(dfs, Mockito.times(2)).recoverLease(FILE);
     Mockito.verify(dfs, Mockito.times(1)).isFileClosed(FILE);
   }
 
-  void testIsSameHdfs(int nnport) throws IOException {
-    try {
-      Class dfsUtilClazz = Class.forName("org.apache.hadoop.hdfs.DFSUtil");
-      dfsUtilClazz.getMethod("getNNServiceRpcAddresses", Configuration.class);
-    } catch (Exception e) {
-      LOG.info("Skip testIsSameHdfs test case because of the no-HA hadoop version.");
-      return;
-    }
-
+  private void testIsSameHdfs(int nnport) throws IOException {
     Configuration conf = HBaseConfiguration.create();
     Path srcPath = new Path("hdfs://localhost:" + nnport + "/");
     Path desPath = new Path("hdfs://127.0.0.1/");
     FileSystem srcFs = srcPath.getFileSystem(conf);
     FileSystem desFs = desPath.getFileSystem(conf);
 
-    assertTrue(FSHDFSUtils.isSameHdfs(conf, srcFs, desFs));
+    assertTrue(FSUtils.isSameHdfs(conf, srcFs, desFs));
 
     desPath = new Path("hdfs://127.0.0.1:8070/");
     desFs = desPath.getFileSystem(conf);
-    assertTrue(!FSHDFSUtils.isSameHdfs(conf, srcFs, desFs));
+    assertTrue(!FSUtils.isSameHdfs(conf, srcFs, desFs));
 
     desPath = new Path("hdfs://127.0.1.1:" + nnport + "/");
     desFs = desPath.getFileSystem(conf);
-    assertTrue(!FSHDFSUtils.isSameHdfs(conf, srcFs, desFs));
+    assertTrue(!FSUtils.isSameHdfs(conf, srcFs, desFs));
 
     conf.set("fs.defaultFS", "hdfs://haosong-hadoop");
     conf.set("dfs.nameservices", "haosong-hadoop");
@@ -139,13 +125,13 @@ public class TestFSHDFSUtils {
     conf.set("dfs.namenode.rpc-address.haosong-hadoop.nn2", "127.10.2.1:8000");
     desPath = new Path("/");
     desFs = desPath.getFileSystem(conf);
-    assertTrue(FSHDFSUtils.isSameHdfs(conf, srcFs, desFs));
+    assertTrue(FSUtils.isSameHdfs(conf, srcFs, desFs));
 
     conf.set("dfs.namenode.rpc-address.haosong-hadoop.nn1", "127.10.2.1:"+nnport);
     conf.set("dfs.namenode.rpc-address.haosong-hadoop.nn2", "127.0.0.1:8000");
     desPath = new Path("/");
     desFs = desPath.getFileSystem(conf);
-    assertTrue(!FSHDFSUtils.isSameHdfs(conf, srcFs, desFs));
+    assertTrue(!FSUtils.isSameHdfs(conf, srcFs, desFs));
   }
 
   @Test
@@ -167,13 +153,13 @@ public class TestFSHDFSUtils {
   /**
    * Version of DFS that has HDFS-4525 in it.
    */
-  static class IsFileClosedDistributedFileSystem extends DistributedFileSystem {
+  private static class IsFileClosedDistributedFileSystem extends DistributedFileSystem {
     /**
      * Close status of a file. Copied over from HDFS-4525
      * @return true if file is already closed
      **/
     @Override
-    public boolean isFileClosed(Path f) throws IOException{
+    public boolean isFileClosed(Path f) throws IOException {
       return false;
     }
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java
index 3b451e3..554cb9d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java
@@ -84,19 +84,20 @@ public class TestFSTableDescriptors {
   @Test
   public void testCreateAndUpdate() throws IOException {
     Path testdir = UTIL.getDataTestDir(name.getMethodName());
-    TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName())).build();
+    TableDescriptor htd =
+      TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName())).build();
     FileSystem fs = FileSystem.get(UTIL.getConfiguration());
     FSTableDescriptors fstd = new FSTableDescriptors(fs, testdir);
     assertTrue(fstd.createTableDescriptor(htd));
     assertFalse(fstd.createTableDescriptor(htd));
-    FileStatus [] statuses = fs.listStatus(testdir);
-    assertTrue("statuses.length="+statuses.length, statuses.length == 1);
+    FileStatus[] statuses = fs.listStatus(testdir);
+    assertTrue("statuses.length=" + statuses.length, statuses.length == 1);
     for (int i = 0; i < 10; i++) {
       fstd.updateTableDescriptor(htd);
     }
     statuses = fs.listStatus(testdir);
     assertTrue(statuses.length == 1);
-    Path tmpTableDir = new Path(FSUtils.getTableDir(testdir, htd.getTableName()), ".tmp");
+    Path tmpTableDir = new Path(CommonFSUtils.getTableDir(testdir, htd.getTableName()), ".tmp");
     statuses = fs.listStatus(tmpTableDir);
     assertTrue(statuses.length == 0);
   }
@@ -104,7 +105,8 @@ public class TestFSTableDescriptors {
   @Test
   public void testSequenceIdAdvancesOnTableInfo() throws IOException {
     Path testdir = UTIL.getDataTestDir(name.getMethodName());
-    TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName())).build();
+    TableDescriptor htd =
+      TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName())).build();
     FileSystem fs = FileSystem.get(UTIL.getConfiguration());
     FSTableDescriptors fstd = new FSTableDescriptors(fs, testdir);
     Path p0 = fstd.updateTableDescriptor(htd);
@@ -432,7 +434,7 @@ public class TestFSTableDescriptors {
   public void testReadingInvalidDirectoryFromFS() throws IOException {
     FileSystem fs = FileSystem.get(UTIL.getConfiguration());
     try {
-      new FSTableDescriptors(fs, FSUtils.getRootDir(UTIL.getConfiguration()))
+      new FSTableDescriptors(fs, CommonFSUtils.getRootDir(UTIL.getConfiguration()))
           .get(TableName.valueOf(HConstants.HBASE_TEMP_DIRECTORY));
       fail("Shouldn't be able to read a table descriptor for the archive directory.");
     } catch (Exception e) {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSUtils.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSUtils.java
index d425557..63c8f86 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSUtils.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSUtils.java
@@ -81,14 +81,17 @@ public class TestFSUtils {
     conf = htu.getConfiguration();
   }
 
-  @Test public void testIsHDFS() throws Exception {
-    assertFalse(FSUtils.isHDFS(conf));
+  @Test
+  public void testIsHDFS() throws Exception {
+    assertFalse(CommonFSUtils.isHDFS(conf));
     MiniDFSCluster cluster = null;
     try {
       cluster = htu.startMiniDFSCluster(1);
-      assertTrue(FSUtils.isHDFS(conf));
+      assertTrue(CommonFSUtils.isHDFS(conf));
     } finally {
-      if (cluster != null) cluster.shutdown();
+      if (cluster != null) {
+        cluster.shutdown();
+      }
     }
   }
 
@@ -204,8 +207,8 @@ public class TestFSUtils {
   }
 
   private void writeVersionFile(Path versionFile, String version) throws IOException {
-    if (FSUtils.isExists(fs, versionFile)) {
-      assertTrue(FSUtils.delete(fs, versionFile, true));
+    if (CommonFSUtils.isExists(fs, versionFile)) {
+      assertTrue(CommonFSUtils.delete(fs, versionFile, true));
     }
     try (FSDataOutputStream s = fs.create(versionFile)) {
       s.writeUTF(version);
@@ -223,13 +226,13 @@ public class TestFSUtils {
     FSUtils.checkVersion(fs, rootdir, true);
     // Now remove the version file and create a metadir so checkVersion fails.
     Path versionFile = new Path(rootdir, HConstants.VERSION_FILE_NAME);
-    assertTrue(FSUtils.isExists(fs, versionFile));
-    assertTrue(FSUtils.delete(fs, versionFile, true));
+    assertTrue(CommonFSUtils.isExists(fs, versionFile));
+    assertTrue(CommonFSUtils.delete(fs, versionFile, true));
     Path metaRegionDir =
         FSUtils.getRegionDirFromRootDir(rootdir, RegionInfoBuilder.FIRST_META_REGIONINFO);
-    FsPermission defaultPerms = FSUtils.getFilePermissions(fs, this.conf,
+    FsPermission defaultPerms = CommonFSUtils.getFilePermissions(fs, this.conf,
         HConstants.DATA_FILE_UMASK_KEY);
-    FSUtils.create(fs, metaRegionDir, defaultPerms, false);
+    CommonFSUtils.create(fs, metaRegionDir, defaultPerms, false);
     boolean thrown = false;
     try {
       FSUtils.checkVersion(fs, rootdir, true);
@@ -268,24 +271,24 @@ public class TestFSUtils {
     final Path rootdir = htu.getDataTestDir();
     final FileSystem fs = rootdir.getFileSystem(conf);
     // default fs permission
-    FsPermission defaultFsPerm = FSUtils.getFilePermissions(fs, conf,
+    FsPermission defaultFsPerm = CommonFSUtils.getFilePermissions(fs, conf,
         HConstants.DATA_FILE_UMASK_KEY);
     // 'hbase.data.umask.enable' is false. We will get default fs permission.
     assertEquals(FsPermission.getFileDefault(), defaultFsPerm);
 
     conf.setBoolean(HConstants.ENABLE_DATA_FILE_UMASK, true);
     // first check that we don't crash if we don't have perms set
-    FsPermission defaultStartPerm = FSUtils.getFilePermissions(fs, conf,
+    FsPermission defaultStartPerm = CommonFSUtils.getFilePermissions(fs, conf,
         HConstants.DATA_FILE_UMASK_KEY);
     // default 'hbase.data.umask'is 000, and this umask will be used when
     // 'hbase.data.umask.enable' is true.
     // Therefore we will not get the real fs default in this case.
     // Instead we will get the starting point FULL_RWX_PERMISSIONS
-    assertEquals(new FsPermission(FSUtils.FULL_RWX_PERMISSIONS), defaultStartPerm);
+    assertEquals(new FsPermission(CommonFSUtils.FULL_RWX_PERMISSIONS), defaultStartPerm);
 
     conf.setStrings(HConstants.DATA_FILE_UMASK_KEY, "077");
     // now check that we get the right perms
-    FsPermission filePerm = FSUtils.getFilePermissions(fs, conf,
+    FsPermission filePerm = CommonFSUtils.getFilePermissions(fs, conf,
         HConstants.DATA_FILE_UMASK_KEY);
     assertEquals(new FsPermission("700"), filePerm);
 
@@ -307,7 +310,7 @@ public class TestFSUtils {
     final Path rootdir = htu.getDataTestDir();
     final FileSystem fs = rootdir.getFileSystem(conf);
     conf.setBoolean(HConstants.ENABLE_DATA_FILE_UMASK, true);
-    FsPermission perms = FSUtils.getFilePermissions(fs, conf, HConstants.DATA_FILE_UMASK_KEY);
+    FsPermission perms = CommonFSUtils.getFilePermissions(fs, conf, HConstants.DATA_FILE_UMASK_KEY);
     // then that the correct file is created
     String file = htu.getRandomUUID().toString();
     Path p = new Path(htu.getDataTestDir(), "temptarget" + File.separator + file);
@@ -315,19 +318,19 @@ public class TestFSUtils {
     try {
       FSDataOutputStream out = FSUtils.create(conf, fs, p, perms, null);
       out.close();
-      assertTrue("The created file should be present", FSUtils.isExists(fs, p));
+      assertTrue("The created file should be present", CommonFSUtils.isExists(fs, p));
       // delete the file with recursion as false. Only the file will be deleted.
-      FSUtils.delete(fs, p, false);
+      CommonFSUtils.delete(fs, p, false);
       // Create another file
       FSDataOutputStream out1 = FSUtils.create(conf, fs, p1, perms, null);
       out1.close();
       // delete the file with recursion as false. Still the file only will be deleted
-      FSUtils.delete(fs, p1, true);
-      assertFalse("The created file should be present", FSUtils.isExists(fs, p1));
+      CommonFSUtils.delete(fs, p1, true);
+      assertFalse("The created file should be present", CommonFSUtils.isExists(fs, p1));
       // and then cleanup
     } finally {
-      FSUtils.delete(fs, p, true);
-      FSUtils.delete(fs, p1, true);
+      CommonFSUtils.delete(fs, p, true);
+      CommonFSUtils.delete(fs, p1, true);
     }
   }
 
@@ -345,7 +348,7 @@ public class TestFSUtils {
   @Test
   public void testRenameAndSetModifyTime() throws Exception {
     MiniDFSCluster cluster = htu.startMiniDFSCluster(1);
-    assertTrue(FSUtils.isHDFS(conf));
+    assertTrue(CommonFSUtils.isHDFS(conf));
 
     FileSystem fs = FileSystem.get(conf);
     Path testDir = htu.getDataTestDirOnTestFS("testArchiveFile");
@@ -355,7 +358,7 @@ public class TestFSUtils {
 
     FSDataOutputStream out = fs.create(p);
     out.close();
-    assertTrue("The created file should be present", FSUtils.isExists(fs, p));
+    assertTrue("The created file should be present", CommonFSUtils.isExists(fs, p));
 
     long expect = System.currentTimeMillis() + 1000;
     assertNotEquals(expect, fs.getFileStatus(p).getModificationTime());
@@ -367,9 +370,9 @@ public class TestFSUtils {
       String dstFile = htu.getRandomUUID().toString();
       Path dst = new Path(testDir , dstFile);
 
-      assertTrue(FSUtils.renameAndSetModifyTime(fs, p, dst));
-      assertFalse("The moved file should not be present", FSUtils.isExists(fs, p));
-      assertTrue("The dst file should be present", FSUtils.isExists(fs, dst));
+      assertTrue(CommonFSUtils.renameAndSetModifyTime(fs, p, dst));
+      assertFalse("The moved file should not be present", CommonFSUtils.isExists(fs, p));
+      assertTrue("The dst file should be present", CommonFSUtils.isExists(fs, dst));
 
       assertEquals(expect, fs.getFileStatus(dst).getModificationTime());
       cluster.shutdown();
@@ -393,15 +396,15 @@ public class TestFSUtils {
     // There should be no exception thrown when setting to default storage policy, which indicates
     // the HDFS API hasn't been called
     try {
-      FSUtils.setStoragePolicy(testFs, new Path("non-exist"), HConstants.DEFAULT_WAL_STORAGE_POLICY,
-        true);
+      CommonFSUtils.setStoragePolicy(testFs, new Path("non-exist"),
+        HConstants.DEFAULT_WAL_STORAGE_POLICY, true);
     } catch (IOException e) {
       Assert.fail("Should have bypassed the FS API when setting default storage policy");
     }
     // There should be exception thrown when given non-default storage policy, which indicates the
     // HDFS API has been called
     try {
-      FSUtils.setStoragePolicy(testFs, new Path("non-exist"), "HOT", true);
+      CommonFSUtils.setStoragePolicy(testFs, new Path("non-exist"), "HOT", true);
       Assert.fail("Should have invoked the FS API but haven't");
     } catch (IOException e) {
       // expected given an invalid path
@@ -436,7 +439,7 @@ public class TestFSUtils {
 
     MiniDFSCluster cluster = htu.startMiniDFSCluster(1);
     try {
-      assertTrue(FSUtils.isHDFS(conf));
+      assertTrue(CommonFSUtils.isHDFS(conf));
 
       FileSystem fs = FileSystem.get(conf);
       Path testDir = htu.getDataTestDirOnTestFS("testArchiveFile");
@@ -444,7 +447,7 @@ public class TestFSUtils {
 
       String storagePolicy =
           conf.get(HConstants.WAL_STORAGE_POLICY, HConstants.DEFAULT_WAL_STORAGE_POLICY);
-      FSUtils.setStoragePolicy(fs, testDir, storagePolicy);
+      CommonFSUtils.setStoragePolicy(fs, testDir, storagePolicy);
 
       String file =htu.getRandomUUID().toString();
       Path p = new Path(testDir, file);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSVisitor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSVisitor.java
index 579362a..7cb3fd4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSVisitor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSVisitor.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hbase.util;
 
 import static org.junit.Assert.assertEquals;
+
 import java.io.IOException;
 import java.util.HashSet;
 import java.util.Set;
@@ -68,7 +69,7 @@ public class TestFSVisitor {
     tableRegions = new HashSet<>();
     tableHFiles = new HashSet<>();
     tableDir = createTableFiles(rootDir, TABLE_NAME, tableRegions, tableFamilies, tableHFiles);
-    FSUtils.logFileSystemState(fs, rootDir, LOG);
+    CommonFSUtils.logFileSystemState(fs, rootDir, LOG);
   }
 
   @After
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 4e4b648..6f175cf 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
@@ -116,7 +116,7 @@ public class TestHBaseFsckMOB extends BaseTestHBaseFsck {
       Path corrupt = new Path(mobFile.getParent(), corruptMobFile);
       TestHFile.truncateFile(fs, mobFile, corrupt);
       LOG.info("Created corrupted mob file " + corrupt);
-      HBaseFsck.debugLsr(conf, FSUtils.getRootDir(conf));
+      HBaseFsck.debugLsr(conf, CommonFSUtils.getRootDir(conf));
       HBaseFsck.debugLsr(conf, MobUtils.getMobHome(conf));
 
       // A corrupt mob file doesn't abort the start of regions, so we can enable the table.
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHFileArchiveUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHFileArchiveUtil.java
index 239b22a..2c7548c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHFileArchiveUtil.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHFileArchiveUtil.java
@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hbase.util;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertNotNull;
 
 import java.io.IOException;
 import org.apache.hadoop.conf.Configuration;
@@ -57,7 +57,7 @@ public class TestHFileArchiveUtil {
   @Test
   public void testGetArchivePath() throws Exception {
     Configuration conf = new Configuration();
-    FSUtils.setRootDir(conf, new Path("root"));
+    CommonFSUtils.setRootDir(conf, new Path("root"));
     assertNotNull(HFileArchiveUtil.getArchivePath(conf));
   }
 
@@ -71,10 +71,10 @@ public class TestHFileArchiveUtil {
   @Test
   public void testGetStoreArchivePath() throws IOException {
       byte[] family = Bytes.toBytes("Family");
-    Path tabledir = FSUtils.getTableDir(rootDir, TableName.valueOf(name.getMethodName()));
+    Path tabledir = CommonFSUtils.getTableDir(rootDir, TableName.valueOf(name.getMethodName()));
     HRegionInfo region = new HRegionInfo(TableName.valueOf(name.getMethodName()));
     Configuration conf = new Configuration();
-    FSUtils.setRootDir(conf, new Path("root"));
+    CommonFSUtils.setRootDir(conf, new Path("root"));
     assertNotNull(HFileArchiveUtil.getStoreArchivePath(conf, region, tabledir, family));
   }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRootPath.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRootPath.java
index c5dc1b2..18c9d42 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRootPath.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRootPath.java
@@ -45,21 +45,21 @@ public class TestRootPath extends TestCase {
   public void testRootPath() {
     try {
       // Try good path
-      FSUtils.validateRootPath(new Path("file:///tmp/hbase/hbase"));
+      CommonFSUtils.validateRootPath(new Path("file:///tmp/hbase/hbase"));
     } catch (IOException e) {
       LOG.error(HBaseMarkers.FATAL, "Unexpected exception checking valid path:", e);
       fail();
     }
     try {
       // Try good path
-      FSUtils.validateRootPath(new Path("hdfs://a:9000/hbase"));
+      CommonFSUtils.validateRootPath(new Path("hdfs://a:9000/hbase"));
     } catch (IOException e) {
       LOG.error(HBaseMarkers.FATAL, "Unexpected exception checking valid path:", e);
       fail();
     }
     try {
       // bad path
-      FSUtils.validateRootPath(new Path("/hbase"));
+      CommonFSUtils.validateRootPath(new Path("/hbase"));
       fail();
     } catch (IOException e) {
       // Expected.
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..5accfce 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
@@ -51,7 +51,7 @@ import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.zookeeper.KeeperException;
 import org.junit.After;
 import org.junit.Before;
@@ -181,9 +181,9 @@ public class OfflineMetaRebuildTestCore {
         TEST_UTIL.getAdmin().unassign(deleteRow, true);
 
         LOG.info("deleting hdfs data: " + hri.toString() + hsa.toString());
-        Path rootDir = FSUtils.getRootDir(conf);
+        Path rootDir = CommonFSUtils.getRootDir(conf);
         FileSystem fs = rootDir.getFileSystem(conf);
-        Path p = new Path(FSUtils.getTableDir(rootDir, htd.getTableName()),
+        Path p = new Path(CommonFSUtils.getTableDir(rootDir, htd.getTableName()),
... 461 lines suppressed ...