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 2022/01/04 15:09:24 UTC

[hbase] branch branch-2 updated (fff59ac -> 53285d8)

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

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


    from fff59ac  HBASE-26642 Increase the timeout for TestStochasticLoadBalancerRegionReplicaLargeCluster (#3995)
     new 04e1980  HBASE-26064 Introduce a StoreFileTracker to abstract the store file tracking logic
     new 826e54c  HBASE-25988 Store the store file list by a file (#3578)
     new e89afdf  HBASE-26079 Use StoreFileTracker when splitting and merging (#3617)
     new f298f9c  HBASE-26224 Introduce a MigrationStoreFileTracker to support migrating from different store file tracker implementations (#3656)
     new 26f263b  HBASE-26246 Persist the StoreFileTracker configurations to TableDescriptor when creating table (#3666)
     new 49c4099  HBASE-26248 Should find a suitable way to let users specify the store file tracker implementation (#3665)
     new dcaea11  HBASE-26264 Add more checks to prevent misconfiguration on store file tracker (#3681)
     new b812a27  HBASE-26280 Use store file tracker when snapshoting (#3685)
     new 030a2a8  HBASE-26326 CreateTableProcedure fails when FileBasedStoreFileTracker… (#3721)
     new ca07fb8  HBASE-26386 Refactor StoreFileTracker implementations to expose the set method (#3774)
     new 80b0422  HBASE-26328 Clone snapshot doesn't load reference files into FILE SFT impl (#3749)
     new c4325ff  HBASE-26263 [Rolling Upgrading] Persist the StoreFileTracker configurations to TableDescriptor for existing tables (#3700)
     new 68252e1  HBASE-26271 Cleanup the broken store files under data directory (#3786)
     new 48e2891  HBASE-26454 CreateTableProcedure still relies on temp dir and renames… (#3845)
     new 53285d8  HBASE-26286: Add support for specifying store file tracker when restoring or cloning snapshot

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


Summary of changes:
 .../java/org/apache/hadoop/hbase/client/Admin.java |  42 +-
 .../org/apache/hadoop/hbase/client/AsyncAdmin.java |  14 +-
 .../hadoop/hbase/client/AsyncHBaseAdmin.java       |   6 +-
 .../hbase/client/ColumnFamilyDescriptor.java       |   5 +
 .../client/ColumnFamilyDescriptorBuilder.java      |   6 +
 .../org/apache/hadoop/hbase/client/HBaseAdmin.java |  26 +-
 .../hadoop/hbase/client/RawAsyncHBaseAdmin.java    |  24 +-
 .../hbase/client/TableDescriptorBuilder.java       |  20 +
 .../src/main/protobuf/Master.proto                 |   1 +
 .../src/main/protobuf/MasterProcedure.proto        |  12 +
 .../{MapReduce.proto => StoreFileTracker.proto}    |  20 +-
 .../java/org/apache/hadoop/hbase/io/HFileLink.java |  63 +-
 .../org/apache/hadoop/hbase/master/HMaster.java    |  31 +-
 .../hadoop/hbase/master/MasterRpcServices.java     |   2 +-
 .../assignment/MergeTableRegionsProcedure.java     |  20 +-
 .../assignment/SplitTableRegionProcedure.java      |  40 +-
 .../hbase/master/migrate/RollingUpgradeChore.java  | 130 ++++
 .../master/procedure/CloneSnapshotProcedure.java   | 106 +--
 .../master/procedure/CreateTableProcedure.java     |  43 +-
 .../master/procedure/DeleteTableProcedure.java     | 115 ++--
 .../procedure/ModifyTableDescriptorProcedure.java  | 161 +++++
 .../master/procedure/ModifyTableProcedure.java     |   5 +
 .../master/procedure/RestoreSnapshotProcedure.java |   7 +-
 .../hbase/master/snapshot/SnapshotManager.java     |  27 +-
 .../hadoop/hbase/mob/DefaultMobStoreCompactor.java |  40 +-
 .../hadoop/hbase/mob/DefaultMobStoreFlusher.java   |   4 +-
 .../regionserver/AbstractMultiFileWriter.java      |   6 +-
 .../hbase/regionserver/BrokenStoreFileCleaner.java | 202 ++++++
 .../regionserver/CreateStoreFileWriterParams.java  | 134 ++++
 .../regionserver/DateTieredMultiFileWriter.java    |   2 +-
 .../hbase/regionserver/DateTieredStoreEngine.java  |   5 +-
 .../hbase/regionserver/DefaultStoreEngine.java     |   5 +-
 .../hbase/regionserver/DefaultStoreFlusher.java    |  11 +-
 .../hadoop/hbase/regionserver/HMobStore.java       |   3 +-
 .../hbase/regionserver/HRegionFileSystem.java      |  50 +-
 .../hadoop/hbase/regionserver/HRegionServer.java   |  26 +
 .../apache/hadoop/hbase/regionserver/HStore.java   | 735 +++++----------------
 .../hadoop/hbase/regionserver/StoreContext.java    |  14 +
 .../hadoop/hbase/regionserver/StoreEngine.java     | 482 +++++++++++++-
 .../hbase/regionserver/StoreFileManager.java       |   9 +
 .../hadoop/hbase/regionserver/StoreFlusher.java    |   9 +-
 .../hadoop/hbase/regionserver/StoreUtils.java      |  37 +-
 .../hbase/regionserver/StripeMultiFileWriter.java  |   2 +-
 .../hbase/regionserver/StripeStoreEngine.java      |   9 +-
 .../hbase/regionserver/StripeStoreFlusher.java     |   9 +-
 .../compactions/AbstractMultiOutputCompactor.java  |  11 +-
 .../hbase/regionserver/compactions/Compactor.java  |  81 ++-
 .../compactions/DateTieredCompactor.java           |   6 +-
 .../regionserver/compactions/DefaultCompactor.java |  25 +-
 .../regionserver/compactions/StripeCompactor.java  |   2 +-
 .../storefiletracker/DefaultStoreFileTracker.java  |  68 ++
 .../FileBasedStoreFileTracker.java                 | 162 +++++
 .../MigrateStoreFileTrackerProcedure.java          |  48 ++
 .../MigrationStoreFileTracker.java                 |  96 +++
 .../storefiletracker/StoreFileListFile.java        | 144 ++++
 .../storefiletracker/StoreFileTracker.java         |  97 +++
 .../storefiletracker/StoreFileTrackerBase.java     | 180 +++++
 .../storefiletracker/StoreFileTrackerFactory.java  | 351 ++++++++++
 .../access/SnapshotScannerHDFSAclHelper.java       |   4 +-
 .../hbase/snapshot/RestoreSnapshotHelper.java      | 104 ++-
 .../hadoop/hbase/snapshot/SnapshotManifest.java    |  42 +-
 .../util/compaction/MajorCompactionRequest.java    |   1 -
 .../org/apache/hadoop/hbase/TestIOFencing.java     |  12 +-
 .../org/apache/hadoop/hbase/client/TestAdmin.java  |   6 +
 .../org/apache/hadoop/hbase/client/TestAdmin3.java |   6 +
 .../hbase/client/TestAsyncTableAdminApi.java       |   7 +
 .../hbase/client/TestAsyncTableAdminApi3.java      |   8 +-
 .../TestCloneSnapshotFromClientCustomSFT.java      |  72 ++
 ...oneSnapshotFromClientCloneLinksAfterDelete.java |   4 +-
 .../hbase/client/TestMobSnapshotFromClient.java    |   7 +-
 .../hbase/client/TestSnapshotFromClient.java       |  34 +-
 .../hadoop/hbase/master/TestMasterFileSystem.java  |  29 +-
 .../migrate/TestMigrateStoreFileTracker.java       | 108 +++
 .../procedure/MasterProcedureTestingUtility.java   |   7 +
 ...=> TestCloneSnapshotProcedureFileBasedSFT.java} |  20 +-
 .../master/procedure/TestCreateTableProcedure.java |  33 +
 .../master/procedure/TestDeleteTableProcedure.java |  66 --
 .../regionserver/TestBrokenStoreFileCleaner.java   | 226 +++++++
 .../regionserver/TestCacheOnWriteInSchema.java     |   6 +-
 .../hbase/regionserver/TestCompactorMemLeak.java   |   4 +-
 .../hbase/regionserver/TestDefaultStoreEngine.java |  10 +-
 .../regionserver/TestDirectStoreSplitsMerges.java  |  32 +-
 .../hadoop/hbase/regionserver/TestHRegion.java     |   4 +-
 .../hadoop/hbase/regionserver/TestHStore.java      |  33 +-
 .../hadoop/hbase/regionserver/TestHStoreFile.java  |  19 +-
 .../regionserver/TestMergesSplitsAddToTracker.java | 267 ++++++++
 .../TestRegionMergeTransactionOnCluster.java       |   6 +-
 .../regionserver/TestStoreFileRefresherChore.java  |   3 +-
 .../regionserver/TestStoreScannerClosure.java      |   6 +-
 .../hbase/regionserver/TestStripeStoreEngine.java  |  13 +-
 .../compactions/TestDateTieredCompactor.java       |  16 +-
 .../compactions/TestStripeCompactionPolicy.java    |  12 +-
 .../compactions/TestStripeCompactor.java           |  16 +-
 .../TestChangeStoreFileTracker.java                | 242 +++++++
 .../TestMigrationStoreFileTracker.java             | 194 ++++++
 .../TestRegionWithFileBasedStoreFileTracker.java   | 108 +++
 .../storefiletracker/TestStoreFileTracker.java     |  61 ++
 .../TestStoreFileTrackerFactory.java               | 110 +++
 .../hbase/snapshot/MobSnapshotTestingUtils.java    |  74 ++-
 .../hbase/snapshot/SnapshotTestingUtils.java       |  16 +-
 hbase-shell/src/main/ruby/hbase/admin.rb           |   4 +-
 hbase-shell/src/main/ruby/hbase_constants.rb       |   1 +
 .../src/main/ruby/shell/commands/clone_snapshot.rb |   6 +-
 .../hadoop/hbase/thrift2/client/ThriftAdmin.java   |   4 +-
 104 files changed, 4796 insertions(+), 1258 deletions(-)
 copy hbase-protocol-shaded/src/main/protobuf/{MapReduce.proto => StoreFileTracker.proto} (74%)
 create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/master/migrate/RollingUpgradeChore.java
 create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableDescriptorProcedure.java
 create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BrokenStoreFileCleaner.java
 create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CreateStoreFileWriterParams.java
 create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/DefaultStoreFileTracker.java
 create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/FileBasedStoreFileTracker.java
 create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrateStoreFileTrackerProcedure.java
 create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java
 create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileListFile.java
 create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java
 create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java
 create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java
 create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClientCustomSFT.java
 create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/master/migrate/TestMigrateStoreFileTracker.java
 copy hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/{TestSCPWithoutMetaWithoutZKCoordinated.java => TestCloneSnapshotProcedureFileBasedSFT.java} (62%)
 create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBrokenStoreFileCleaner.java
 create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMergesSplitsAddToTracker.java
 create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestChangeStoreFileTracker.java
 create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestMigrationStoreFileTracker.java
 create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestRegionWithFileBasedStoreFileTracker.java
 create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTracker.java
 create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTrackerFactory.java

[hbase] 07/15: HBASE-26264 Add more checks to prevent misconfiguration on store file tracker (#3681)

Posted by zh...@apache.org.
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

commit dcaea11a520fc754e4c6d3dfc9aea78f8b7bd57b
Author: Duo Zhang <zh...@apache.org>
AuthorDate: Wed Sep 15 23:00:03 2021 +0800

    HBASE-26264 Add more checks to prevent misconfiguration on store file tracker (#3681)
    
    Signed-off-by: Josh Elser <el...@apache.org>
---
 .../assignment/MergeTableRegionsProcedure.java     |   3 +-
 .../assignment/SplitTableRegionProcedure.java      |   3 +-
 .../master/procedure/CreateTableProcedure.java     |   8 +-
 .../master/procedure/ModifyTableProcedure.java     |   5 +
 .../hbase/regionserver/HRegionFileSystem.java      |   2 +-
 .../MigrationStoreFileTracker.java                 |   8 +
 .../storefiletracker/StoreFileTrackerFactory.java  | 173 +++++++++++++--
 .../TestChangeStoreFileTracker.java                | 242 +++++++++++++++++++++
 8 files changed, 422 insertions(+), 22 deletions(-)

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 75844991..6e5151c 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
@@ -615,8 +615,7 @@ public class MergeTableRegionsProcedure
       String family = hcd.getNameAsString();
       Configuration trackerConfig =
         StoreFileTrackerFactory.mergeConfigurations(env.getMasterConfiguration(), htd, hcd);
-      StoreFileTracker tracker = StoreFileTrackerFactory.create(trackerConfig, true,
-        family, regionFs);
+      StoreFileTracker tracker = StoreFileTrackerFactory.create(trackerConfig, family, regionFs);
       final Collection<StoreFileInfo> storeFiles = tracker.load();
       if (storeFiles != null && storeFiles.size() > 0) {
         final Configuration storeConfiguration =
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 ff16dc5..aa0c938 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
@@ -670,8 +670,7 @@ public class SplitTableRegionProcedure
       String family = cfd.getNameAsString();
       Configuration trackerConfig = StoreFileTrackerFactory.
         mergeConfigurations(env.getMasterConfiguration(), htd, htd.getColumnFamily(cfd.getName()));
-      StoreFileTracker tracker = StoreFileTrackerFactory.create(trackerConfig, true,
-        family, regionFs);
+      StoreFileTracker tracker = StoreFileTrackerFactory.create(trackerConfig, family, regionFs);
       Collection<StoreFileInfo> sfis = tracker.load();
       if (sfis == null) {
         continue;
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 219821f..80ed96a 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
@@ -257,15 +257,17 @@ public class CreateTableProcedure
       return false;
     }
 
+    // check for store file tracker configurations
+    StoreFileTrackerFactory.checkForCreateTable(env.getMasterConfiguration(), tableDescriptor);
+
     return true;
   }
 
   private void preCreate(final MasterProcedureEnv env)
       throws IOException, InterruptedException {
     if (!getTableName().isSystemTable()) {
-      ProcedureSyncWait.getMasterQuotaManager(env)
-        .checkNamespaceTableAndRegionQuota(
-          getTableName(), (newRegions != null ? newRegions.size() : 0));
+      ProcedureSyncWait.getMasterQuotaManager(env).checkNamespaceTableAndRegionQuota(getTableName(),
+        (newRegions != null ? newRegions.size() : 0));
     }
 
     TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableDescriptor);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java
index 9892dba..e58fc92 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.zksyncer.MetaLocationSyncer;
 import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
+import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
@@ -317,6 +318,10 @@ public class ModifyTableProcedure
 
     this.deleteColumnFamilyInModify = isDeleteColumnFamily(unmodifiedTableDescriptor,
       modifiedTableDescriptor);
+
+    // check for store file tracker configurations
+    StoreFileTrackerFactory.checkForModifyTable(env.getMasterConfiguration(),
+      unmodifiedTableDescriptor, modifiedTableDescriptor);
   }
 
   /**
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 aa0ee27..e78d8ad 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
@@ -629,7 +629,7 @@ public class HRegionFileSystem {
         Configuration config = StoreFileTrackerFactory.mergeConfigurations(conf, tblDesc,
           tblDesc.getColumnFamily(Bytes.toBytes(familyName)));
         return StoreFileTrackerFactory.
-          create(config, true, familyName, regionFs);
+          create(config, familyName, regionFs);
       });
       fileInfoMap.computeIfAbsent(familyName, l -> new ArrayList<>());
       List<StoreFileInfo> infos = fileInfoMap.get(familyName);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java
index 3eeef90..1946d4b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java
@@ -98,4 +98,12 @@ class MigrationStoreFileTracker extends StoreFileTrackerBase {
       builder.setValue(DST_IMPL, dst.getTrackerName());
     }
   }
+
+  static Class<? extends StoreFileTracker> getSrcTrackerClass(Configuration conf) {
+    return StoreFileTrackerFactory.getStoreFileTrackerClassForMigration(conf, SRC_IMPL);
+  }
+
+  static Class<? extends StoreFileTracker> getDstTrackerClass(Configuration conf) {
+    return StoreFileTrackerFactory.getStoreFileTrackerClassForMigration(conf, DST_IMPL);
+  }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java
index 9be19ec..90704fe 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java
@@ -15,10 +15,12 @@
  */
 package org.apache.hadoop.hbase.regionserver.storefiletracker;
 
+import java.io.IOException;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.TableDescriptor;
@@ -111,13 +113,13 @@ public final class StoreFileTrackerFactory {
    * Used at master side when splitting/merging regions, as we do not have a Store, thus no
    * StoreContext at master side.
    */
-  public static StoreFileTracker create(Configuration conf, boolean isPrimaryReplica, String family,
+  public static StoreFileTracker create(Configuration conf, String family,
     HRegionFileSystem regionFs) {
     ColumnFamilyDescriptorBuilder fDescBuilder =
       ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(family));
     StoreContext ctx = StoreContext.getBuilder().withColumnFamilyDescriptor(fDescBuilder.build())
       .withRegionFileSystem(regionFs).build();
-    return StoreFileTrackerFactory.create(conf, isPrimaryReplica, ctx);
+    return StoreFileTrackerFactory.create(conf, true, ctx);
   }
 
   public static Configuration mergeConfigurations(Configuration global, TableDescriptor table,
@@ -125,30 +127,35 @@ public final class StoreFileTrackerFactory {
     return StoreUtils.createStoreConfiguration(global, table, family);
   }
 
-  /**
-   * Create store file tracker to be used as source or destination for
-   * {@link MigrationStoreFileTracker}.
-   */
-  static StoreFileTrackerBase createForMigration(Configuration conf, String configName,
-    boolean isPrimaryReplica, StoreContext ctx) {
+  static Class<? extends StoreFileTrackerBase>
+    getStoreFileTrackerClassForMigration(Configuration conf, String configName) {
     String trackerName =
       Preconditions.checkNotNull(conf.get(configName), "config %s is not set", configName);
-    Class<? extends StoreFileTrackerBase> tracker;
     try {
-      tracker =
-        Trackers.valueOf(trackerName.toUpperCase()).clazz.asSubclass(StoreFileTrackerBase.class);
+      return Trackers.valueOf(trackerName.toUpperCase()).clazz
+        .asSubclass(StoreFileTrackerBase.class);
     } catch (IllegalArgumentException e) {
       // Fall back to them specifying a class name
       try {
-        tracker = Class.forName(trackerName).asSubclass(StoreFileTrackerBase.class);
+        return Class.forName(trackerName).asSubclass(StoreFileTrackerBase.class);
       } catch (ClassNotFoundException cnfe) {
         throw new RuntimeException(cnfe);
       }
     }
+  }
+
+  /**
+   * Create store file tracker to be used as source or destination for
+   * {@link MigrationStoreFileTracker}.
+   */
+  static StoreFileTrackerBase createForMigration(Configuration conf, String configName,
+    boolean isPrimaryReplica, StoreContext ctx) {
+    Class<? extends StoreFileTrackerBase> tracker =
+      getStoreFileTrackerClassForMigration(conf, configName);
     // prevent nest of MigrationStoreFileTracker, it will cause infinite recursion.
     if (MigrationStoreFileTracker.class.isAssignableFrom(tracker)) {
-      throw new IllegalArgumentException("Should not specify " + configName + " as " +
-        Trackers.MIGRATION + " because it can not be nested");
+      throw new IllegalArgumentException("Should not specify " + configName + " as "
+        + Trackers.MIGRATION + " because it can not be nested");
     }
     LOG.info("instantiating StoreFileTracker impl {} as {}", tracker.getName(), configName);
     return ReflectionUtils.newInstance(tracker, conf, isPrimaryReplica, ctx);
@@ -161,4 +168,142 @@ public final class StoreFileTrackerFactory {
     StoreFileTracker tracker = StoreFileTrackerFactory.create(conf, true, context);
     tracker.persistConfiguration(builder);
   }
+
+  // should not use MigrationStoreFileTracker for new family
+  private static void checkForNewFamily(Configuration conf, TableDescriptor table,
+    ColumnFamilyDescriptor family) throws IOException {
+    Configuration mergedConf = mergeConfigurations(conf, table, family);
+    Class<? extends StoreFileTracker> tracker = getTrackerClass(mergedConf);
+    if (MigrationStoreFileTracker.class.isAssignableFrom(tracker)) {
+      throw new DoNotRetryIOException(
+        "Should not use " + Trackers.MIGRATION + " as store file tracker for new family "
+          + family.getNameAsString() + " of table " + table.getTableName());
+    }
+  }
+
+  /**
+   * Pre check when creating a new table.
+   * <p/>
+   * For now, only make sure that we do not use {@link Trackers#MIGRATION} for newly created tables.
+   * @throws IOException when there are check errors, the upper layer should fail the
+   *           {@code CreateTableProcedure}.
+   */
+  public static void checkForCreateTable(Configuration conf, TableDescriptor table)
+    throws IOException {
+    for (ColumnFamilyDescriptor family : table.getColumnFamilies()) {
+      checkForNewFamily(conf, table, family);
+    }
+  }
+
+
+  /**
+   * Pre check when modifying a table.
+   * <p/>
+   * The basic idea is when you want to change the store file tracker implementation, you should use
+   * {@link Trackers#MIGRATION} first and then change to the destination store file tracker
+   * implementation.
+   * <p/>
+   * There are several rules:
+   * <ul>
+   * <li>For newly added family, you should not use {@link Trackers#MIGRATION}.</li>
+   * <li>For modifying a family:
+   * <ul>
+   * <li>If old tracker is {@link Trackers#MIGRATION}, then:
+   * <ul>
+   * <li>The new tracker is also {@link Trackers#MIGRATION}, then they must have the same src and
+   * dst tracker.</li>
+   * <li>The new tracker is not {@link Trackers#MIGRATION}, then the new tracker must be the dst
+   * tracker of the old tracker.</li>
+   * </ul>
+   * </li>
+   * <li>If the old tracker is not {@link Trackers#MIGRATION}, then:
+   * <ul>
+   * <li>If the new tracker is {@link Trackers#MIGRATION}, then the old tracker must be the src
+   * tracker of the new tracker.</li>
+   * <li>If the new tracker is not {@link Trackers#MIGRATION}, then the new tracker must be the same
+   * with old tracker.</li>
+   * </ul>
+   * </li>
+   * </ul>
+   * </li>
+   * </ul>
+   * @throws IOException when there are check errors, the upper layer should fail the
+   *           {@code ModifyTableProcedure}.
+   */
+  public static void checkForModifyTable(Configuration conf, TableDescriptor oldTable,
+    TableDescriptor newTable) throws IOException {
+    for (ColumnFamilyDescriptor newFamily : newTable.getColumnFamilies()) {
+      ColumnFamilyDescriptor oldFamily = oldTable.getColumnFamily(newFamily.getName());
+      if (oldFamily == null) {
+        checkForNewFamily(conf, newTable, newFamily);
+        continue;
+      }
+      Configuration oldConf = mergeConfigurations(conf, oldTable, oldFamily);
+      Configuration newConf = mergeConfigurations(conf, newTable, newFamily);
+
+      Class<? extends StoreFileTracker> oldTracker = getTrackerClass(oldConf);
+      Class<? extends StoreFileTracker> newTracker = getTrackerClass(newConf);
+
+      if (MigrationStoreFileTracker.class.isAssignableFrom(oldTracker)) {
+        Class<? extends StoreFileTracker> oldSrcTracker =
+          MigrationStoreFileTracker.getSrcTrackerClass(oldConf);
+        Class<? extends StoreFileTracker> oldDstTracker =
+          MigrationStoreFileTracker.getDstTrackerClass(oldConf);
+        if (oldTracker.equals(newTracker)) {
+          // confirm that we have the same src tracker and dst tracker
+          Class<? extends StoreFileTracker> newSrcTracker =
+            MigrationStoreFileTracker.getSrcTrackerClass(newConf);
+          if (!oldSrcTracker.equals(newSrcTracker)) {
+            throw new DoNotRetryIOException(
+              "The src tracker has been changed from " + getStoreFileTrackerName(oldSrcTracker)
+                + " to " + getStoreFileTrackerName(newSrcTracker) + " for family "
+                + newFamily.getNameAsString() + " of table " + newTable.getTableName());
+          }
+          Class<? extends StoreFileTracker> newDstTracker =
+            MigrationStoreFileTracker.getDstTrackerClass(newConf);
+          if (!oldDstTracker.equals(newDstTracker)) {
+            throw new DoNotRetryIOException(
+              "The dst tracker has been changed from " + getStoreFileTrackerName(oldDstTracker)
+                + " to " + getStoreFileTrackerName(newDstTracker) + " for family "
+                + newFamily.getNameAsString() + " of table " + newTable.getTableName());
+          }
+        } else {
+          // we can only change to the dst tracker
+          if (!newTracker.equals(oldDstTracker)) {
+            throw new DoNotRetryIOException(
+              "Should migrate tracker to " + getStoreFileTrackerName(oldDstTracker) + " but got "
+                + getStoreFileTrackerName(newTracker) + " for family " + newFamily.getNameAsString()
+                + " of table " + newTable.getTableName());
+          }
+        }
+      } else {
+        if (!oldTracker.equals(newTracker)) {
+          // can only change to MigrationStoreFileTracker and the src tracker should be the old
+          // tracker
+          if (!MigrationStoreFileTracker.class.isAssignableFrom(newTracker)) {
+            throw new DoNotRetryIOException("Should change to " + Trackers.MIGRATION
+              + " first when migrating from " + getStoreFileTrackerName(oldTracker) + " for family "
+              + newFamily.getNameAsString() + " of table " + newTable.getTableName());
+          }
+          Class<? extends StoreFileTracker> newSrcTracker =
+            MigrationStoreFileTracker.getSrcTrackerClass(newConf);
+          if (!oldTracker.equals(newSrcTracker)) {
+            throw new DoNotRetryIOException(
+              "Should use src tracker " + getStoreFileTrackerName(oldTracker) + " first but got "
+                + getStoreFileTrackerName(newSrcTracker) + " when migrating from "
+                + getStoreFileTrackerName(oldTracker) + " for family " + newFamily.getNameAsString()
+                + " of table " + newTable.getTableName());
+          }
+          Class<? extends StoreFileTracker> newDstTracker =
+            MigrationStoreFileTracker.getDstTrackerClass(newConf);
+          // the src and dst tracker should not be the same
+          if (newSrcTracker.equals(newDstTracker)) {
+            throw new DoNotRetryIOException("The src tracker and dst tracker are both "
+              + getStoreFileTrackerName(newSrcTracker) + " for family "
+              + newFamily.getNameAsString() + " of table " + newTable.getTableName());
+          }
+        }
+      }
+    }
+  }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestChangeStoreFileTracker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestChangeStoreFileTracker.java
new file mode 100644
index 0000000..110f896
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestChangeStoreFileTracker.java
@@ -0,0 +1,242 @@
+/**
+ * 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.regionserver.storefiletracker;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.TableNameTestRule;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+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.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.hbase.thirdparty.com.google.common.collect.Iterables;
+
+/**
+ * Test changing store file tracker implementation by altering table.
+ */
+@Category({ RegionServerTests.class, MediumTests.class })
+public class TestChangeStoreFileTracker {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestChangeStoreFileTracker.class);
+
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  @Rule
+  public final TableNameTestRule tableName = new TableNameTestRule();
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    UTIL.startMiniCluster(1);
+  }
+
+  @AfterClass
+  public static void tearDown() throws IOException {
+    UTIL.shutdownMiniCluster();
+  }
+
+  @Test(expected = DoNotRetryIOException.class)
+  public void testCreateError() throws IOException {
+    TableDescriptor td = TableDescriptorBuilder.newBuilder(tableName.getTableName())
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of("family"))
+      .setValue(StoreFileTrackerFactory.TRACKER_IMPL,
+        StoreFileTrackerFactory.Trackers.MIGRATION.name())
+      .setValue(MigrationStoreFileTracker.SRC_IMPL, StoreFileTrackerFactory.Trackers.DEFAULT.name())
+      .setValue(MigrationStoreFileTracker.DST_IMPL, StoreFileTrackerFactory.Trackers.FILE.name())
+      .build();
+    UTIL.getAdmin().createTable(td);
+  }
+
+  @Test(expected = DoNotRetryIOException.class)
+  public void testModifyError1() throws IOException {
+    TableDescriptor td = TableDescriptorBuilder.newBuilder(tableName.getTableName())
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of("family")).build();
+    UTIL.getAdmin().createTable(td);
+    TableDescriptor newTd = TableDescriptorBuilder.newBuilder(td)
+      .setValue(StoreFileTrackerFactory.TRACKER_IMPL, StoreFileTrackerFactory.Trackers.FILE.name())
+      .build();
+    UTIL.getAdmin().modifyTable(newTd);
+  }
+
+  @Test(expected = DoNotRetryIOException.class)
+  public void testModifyError2() throws IOException {
+    TableDescriptor td = TableDescriptorBuilder.newBuilder(tableName.getTableName())
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of("family")).build();
+    UTIL.getAdmin().createTable(td);
+    TableDescriptor newTd = TableDescriptorBuilder.newBuilder(td)
+      .setValue(StoreFileTrackerFactory.TRACKER_IMPL,
+        StoreFileTrackerFactory.Trackers.MIGRATION.name())
+      .setValue(MigrationStoreFileTracker.SRC_IMPL, StoreFileTrackerFactory.Trackers.FILE.name())
+      .setValue(MigrationStoreFileTracker.DST_IMPL, StoreFileTrackerFactory.Trackers.DEFAULT.name())
+      .build();
+    UTIL.getAdmin().modifyTable(newTd);
+  }
+
+  @Test(expected = DoNotRetryIOException.class)
+  public void testModifyError3() throws IOException {
+    TableDescriptor td = TableDescriptorBuilder.newBuilder(tableName.getTableName())
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of("family")).build();
+    UTIL.getAdmin().createTable(td);
+    TableDescriptor newTd = TableDescriptorBuilder.newBuilder(td)
+      .setValue(StoreFileTrackerFactory.TRACKER_IMPL,
+        StoreFileTrackerFactory.Trackers.MIGRATION.name())
+      .setValue(MigrationStoreFileTracker.SRC_IMPL, StoreFileTrackerFactory.Trackers.DEFAULT.name())
+      .setValue(MigrationStoreFileTracker.DST_IMPL, StoreFileTrackerFactory.Trackers.DEFAULT.name())
+      .build();
+    UTIL.getAdmin().modifyTable(newTd);
+  }
+
+  // return the TableDescriptor for creating table
+  private TableDescriptor createTableAndChangeToMigrationTracker() throws IOException {
+    TableDescriptor td = TableDescriptorBuilder.newBuilder(tableName.getTableName())
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of("family")).build();
+    UTIL.getAdmin().createTable(td);
+    TableDescriptor newTd = TableDescriptorBuilder.newBuilder(td)
+      .setValue(StoreFileTrackerFactory.TRACKER_IMPL,
+        StoreFileTrackerFactory.Trackers.MIGRATION.name())
+      .setValue(MigrationStoreFileTracker.SRC_IMPL, StoreFileTrackerFactory.Trackers.DEFAULT.name())
+      .setValue(MigrationStoreFileTracker.DST_IMPL, StoreFileTrackerFactory.Trackers.FILE.name())
+      .build();
+    UTIL.getAdmin().modifyTable(newTd);
+    return td;
+  }
+
+  @Test(expected = DoNotRetryIOException.class)
+  public void testModifyError4() throws IOException {
+    TableDescriptor td = createTableAndChangeToMigrationTracker();
+    TableDescriptor newTd = TableDescriptorBuilder.newBuilder(td)
+      .setValue(StoreFileTrackerFactory.TRACKER_IMPL,
+        StoreFileTrackerFactory.Trackers.MIGRATION.name())
+      .setValue(MigrationStoreFileTracker.SRC_IMPL, StoreFileTrackerFactory.Trackers.FILE.name())
+      .setValue(MigrationStoreFileTracker.DST_IMPL, StoreFileTrackerFactory.Trackers.DEFAULT.name())
+      .build();
+    UTIL.getAdmin().modifyTable(newTd);
+  }
+
+  @Test(expected = DoNotRetryIOException.class)
+  public void testModifyError5() throws IOException {
+    TableDescriptor td = createTableAndChangeToMigrationTracker();
+    TableDescriptor newTd = TableDescriptorBuilder.newBuilder(td)
+      .setValue(StoreFileTrackerFactory.TRACKER_IMPL,
+        StoreFileTrackerFactory.Trackers.MIGRATION.name())
+      .setValue(MigrationStoreFileTracker.SRC_IMPL, StoreFileTrackerFactory.Trackers.DEFAULT.name())
+      .setValue(MigrationStoreFileTracker.DST_IMPL, StoreFileTrackerFactory.Trackers.DEFAULT.name())
+      .build();
+    UTIL.getAdmin().modifyTable(newTd);
+  }
+
+  @Test(expected = DoNotRetryIOException.class)
+  public void testModifyError6() throws IOException {
+    TableDescriptor td = createTableAndChangeToMigrationTracker();
+    TableDescriptor newTd =
+      TableDescriptorBuilder.newBuilder(td).setValue(StoreFileTrackerFactory.TRACKER_IMPL,
+        StoreFileTrackerFactory.Trackers.DEFAULT.name()).build();
+    UTIL.getAdmin().modifyTable(newTd);
+  }
+
+  @Test(expected = DoNotRetryIOException.class)
+  public void testModifyError7() throws IOException {
+    TableDescriptor td = TableDescriptorBuilder.newBuilder(tableName.getTableName())
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of("family")).build();
+    UTIL.getAdmin().createTable(td);
+    TableDescriptor newTd = TableDescriptorBuilder.newBuilder(tableName.getTableName())
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of("family"))
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("family1"))
+        .setConfiguration(StoreFileTrackerFactory.TRACKER_IMPL,
+          StoreFileTrackerFactory.Trackers.MIGRATION.name())
+        .build())
+      .build();
+    UTIL.getAdmin().modifyTable(newTd);
+  }
+
+  // actually a NPE as we do not specify the src and dst impl for migration store file tracker
+  @Test(expected = IOException.class)
+  public void testModifyError8() throws IOException {
+    TableDescriptor td = TableDescriptorBuilder.newBuilder(tableName.getTableName())
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of("family")).build();
+    UTIL.getAdmin().createTable(td);
+    TableDescriptor newTd =
+      TableDescriptorBuilder.newBuilder(td).setValue(StoreFileTrackerFactory.TRACKER_IMPL,
+        StoreFileTrackerFactory.Trackers.MIGRATION.name()).build();
+    UTIL.getAdmin().modifyTable(newTd);
+  }
+
+  private String getStoreFileName(TableName table, byte[] family) {
+    return Iterables
+      .getOnlyElement(Iterables.getOnlyElement(UTIL.getMiniHBaseCluster().getRegions(table))
+        .getStore(family).getStorefiles())
+      .getPath().getName();
+  }
+
+  @Test
+  public void testModify() throws IOException {
+    TableName tn = tableName.getTableName();
+    byte[] row = Bytes.toBytes("row");
+    byte[] family = Bytes.toBytes("family");
+    byte[] qualifier = Bytes.toBytes("qualifier");
+    byte[] value = Bytes.toBytes("value");
+    TableDescriptor td = TableDescriptorBuilder.newBuilder(tn)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(family)).build();
+    UTIL.getAdmin().createTable(td);
+    try (Table table = UTIL.getConnection().getTable(tn)) {
+      table.put(new Put(row).addColumn(family, qualifier, value));
+    }
+    UTIL.flush(tn);
+    String fileName = getStoreFileName(tn, family);
+
+    TableDescriptor newTd = TableDescriptorBuilder.newBuilder(td)
+      .setValue(StoreFileTrackerFactory.TRACKER_IMPL,
+        StoreFileTrackerFactory.Trackers.MIGRATION.name())
+      .setValue(MigrationStoreFileTracker.SRC_IMPL, StoreFileTrackerFactory.Trackers.DEFAULT.name())
+      .setValue(MigrationStoreFileTracker.DST_IMPL, StoreFileTrackerFactory.Trackers.FILE.name())
+      .build();
+    UTIL.getAdmin().modifyTable(newTd);
+    assertEquals(fileName, getStoreFileName(tn, family));
+    try (Table table = UTIL.getConnection().getTable(tn)) {
+      assertArrayEquals(value, table.get(new Get(row)).getValue(family, qualifier));
+    }
+
+    TableDescriptor newTd2 = TableDescriptorBuilder.newBuilder(td)
+      .setValue(StoreFileTrackerFactory.TRACKER_IMPL, StoreFileTrackerFactory.Trackers.FILE.name())
+      .build();
+    UTIL.getAdmin().modifyTable(newTd2);
+    assertEquals(fileName, getStoreFileName(tn, family));
+    try (Table table = UTIL.getConnection().getTable(tn)) {
+      assertArrayEquals(value, table.get(new Get(row)).getValue(family, qualifier));
+    }
+  }
+}

[hbase] 03/15: HBASE-26079 Use StoreFileTracker when splitting and merging (#3617)

Posted by zh...@apache.org.
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

commit e89afdfa0823fa2149c9f7cfb0b8fb80cefeaedb
Author: Wellington Ramos Chevreuil <wc...@apache.org>
AuthorDate: Wed Sep 8 10:31:49 2021 +0100

    HBASE-26079 Use StoreFileTracker when splitting and merging (#3617)
    
    Signed-off-by: Duo Zhang <zh...@apache.org>
---
 .../assignment/MergeTableRegionsProcedure.java     |  22 +-
 .../assignment/SplitTableRegionProcedure.java      |  42 ++--
 .../hbase/regionserver/HRegionFileSystem.java      |  42 +++-
 .../storefiletracker/DefaultStoreFileTracker.java  |   4 +-
 .../storefiletracker/StoreFileTracker.java         |   1 -
 .../storefiletracker/StoreFileTrackerFactory.java  |  33 ++-
 .../hbase/regionserver/TestDefaultStoreEngine.java |   1 +
 .../regionserver/TestDirectStoreSplitsMerges.java  |  32 ++-
 .../hadoop/hbase/regionserver/TestHStoreFile.java  |  19 +-
 .../regionserver/TestMergesSplitsAddToTracker.java | 267 +++++++++++++++++++++
 .../hbase/regionserver/TestStripeStoreEngine.java  |   1 +
 .../storefiletracker/TestStoreFileTracker.java}    |  42 ++--
 12 files changed, 439 insertions(+), 67 deletions(-)

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 347c7d2..75844991 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.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -57,6 +56,8 @@ 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.regionserver.StoreUtils;
+import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTracker;
+import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.wal.WALSplitUtil;
@@ -588,30 +589,35 @@ public class MergeTableRegionsProcedure
     final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
     final Path tableDir = CommonFSUtils.getTableDir(mfs.getRootDir(), regionsToMerge[0].getTable());
     final FileSystem fs = mfs.getFileSystem();
-
+    List<Path> mergedFiles = new ArrayList<>();
     HRegionFileSystem mergeRegionFs = HRegionFileSystem.createRegionOnFileSystem(
       env.getMasterConfiguration(), fs, tableDir, mergedRegion);
 
     for (RegionInfo ri: this.regionsToMerge) {
       HRegionFileSystem regionFs = HRegionFileSystem.openRegionFromFileSystem(
           env.getMasterConfiguration(), fs, tableDir, ri, false);
-      mergeStoreFiles(env, regionFs, mergeRegionFs, mergedRegion);
+      mergedFiles.addAll(mergeStoreFiles(env, regionFs, mergeRegionFs, mergedRegion));
     }
     assert mergeRegionFs != null;
-    mergeRegionFs.commitMergedRegion();
+    mergeRegionFs.commitMergedRegion(mergedFiles, env);
 
     // Prepare to create merged regions
     env.getAssignmentManager().getRegionStates().
         getOrCreateRegionStateNode(mergedRegion).setState(State.MERGING_NEW);
   }
 
-  private void mergeStoreFiles(MasterProcedureEnv env, HRegionFileSystem regionFs,
+  private List<Path> mergeStoreFiles(MasterProcedureEnv env, HRegionFileSystem regionFs,
     HRegionFileSystem mergeRegionFs, RegionInfo mergedRegion) throws IOException {
     final TableDescriptor htd = env.getMasterServices().getTableDescriptors()
       .get(mergedRegion.getTable());
+    List<Path> mergedFiles = new ArrayList<>();
     for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) {
       String family = hcd.getNameAsString();
-      final Collection<StoreFileInfo> storeFiles = regionFs.getStoreFiles(family);
+      Configuration trackerConfig =
+        StoreFileTrackerFactory.mergeConfigurations(env.getMasterConfiguration(), htd, hcd);
+      StoreFileTracker tracker = StoreFileTrackerFactory.create(trackerConfig, true,
+        family, regionFs);
+      final Collection<StoreFileInfo> storeFiles = tracker.load();
       if (storeFiles != null && storeFiles.size() > 0) {
         final Configuration storeConfiguration =
           StoreUtils.createStoreConfiguration(env.getMasterConfiguration(), htd, hcd);
@@ -623,11 +629,13 @@ public class MergeTableRegionsProcedure
           // is running in a regionserver's Store context, or we might not be able
           // to read the hfiles.
           storeFileInfo.setConf(storeConfiguration);
-          mergeRegionFs.mergeStoreFile(regionFs.getRegionInfo(), family,
+          Path refFile = mergeRegionFs.mergeStoreFile(regionFs.getRegionInfo(), family,
             new HStoreFile(storeFileInfo, hcd.getBloomFilterType(), CacheConfig.DISABLED));
+          mergedFiles.add(refFile);
         }
       }
     }
+    return mergedFiles;
   }
 
   /**
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 fbd8729..ff16dc5 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;
@@ -66,6 +65,8 @@ import org.apache.hadoop.hbase.regionserver.RegionSplitPolicy;
 import org.apache.hadoop.hbase.regionserver.RegionSplitRestriction;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
 import org.apache.hadoop.hbase.regionserver.StoreUtils;
+import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTracker;
+import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
@@ -621,21 +622,20 @@ public class SplitTableRegionProcedure
     final FileSystem fs = mfs.getFileSystem();
     HRegionFileSystem regionFs = HRegionFileSystem.openRegionFromFileSystem(
       env.getMasterConfiguration(), fs, tabledir, getParentRegion(), false);
-
     regionFs.createSplitsDir(daughterOneRI, daughterTwoRI);
 
-    Pair<Integer, Integer> expectedReferences = splitStoreFiles(env, regionFs);
+    Pair<List<Path>, List<Path>> expectedReferences = splitStoreFiles(env, regionFs);
 
-    assertSplitResultFilesCount(fs, expectedReferences.getFirst(),
+    assertSplitResultFilesCount(fs, expectedReferences.getFirst().size(),
       regionFs.getSplitsDir(daughterOneRI));
-    regionFs.commitDaughterRegion(daughterOneRI);
-    assertSplitResultFilesCount(fs, expectedReferences.getFirst(),
+    regionFs.commitDaughterRegion(daughterOneRI, expectedReferences.getFirst(), env);
+    assertSplitResultFilesCount(fs, expectedReferences.getFirst().size(),
       new Path(tabledir, daughterOneRI.getEncodedName()));
 
-    assertSplitResultFilesCount(fs, expectedReferences.getSecond(),
+    assertSplitResultFilesCount(fs, expectedReferences.getSecond().size(),
       regionFs.getSplitsDir(daughterTwoRI));
-    regionFs.commitDaughterRegion(daughterTwoRI);
-    assertSplitResultFilesCount(fs, expectedReferences.getSecond(),
+    regionFs.commitDaughterRegion(daughterTwoRI, expectedReferences.getSecond(), env);
+    assertSplitResultFilesCount(fs, expectedReferences.getSecond().size(),
       new Path(tabledir, daughterTwoRI.getEncodedName()));
   }
 
@@ -652,7 +652,7 @@ public class SplitTableRegionProcedure
    * Create Split directory
    * @param env MasterProcedureEnv
    */
-  private Pair<Integer, Integer> splitStoreFiles(final MasterProcedureEnv env,
+  private Pair<List<Path>, List<Path>> splitStoreFiles(final MasterProcedureEnv env,
       final HRegionFileSystem regionFs) throws IOException {
     final Configuration conf = env.getMasterConfiguration();
     TableDescriptor htd = env.getMasterServices().getTableDescriptors().get(getTableName());
@@ -668,7 +668,11 @@ public class SplitTableRegionProcedure
         new HashMap<String, Collection<StoreFileInfo>>(htd.getColumnFamilyCount());
     for (ColumnFamilyDescriptor cfd : htd.getColumnFamilies()) {
       String family = cfd.getNameAsString();
-      Collection<StoreFileInfo> sfis = regionFs.getStoreFiles(family);
+      Configuration trackerConfig = StoreFileTrackerFactory.
+        mergeConfigurations(env.getMasterConfiguration(), htd, htd.getColumnFamily(cfd.getName()));
+      StoreFileTracker tracker = StoreFileTrackerFactory.create(trackerConfig, true,
+        family, regionFs);
+      Collection<StoreFileInfo> sfis = tracker.load();
       if (sfis == null) {
         continue;
       }
@@ -694,7 +698,7 @@ public class SplitTableRegionProcedure
     }
     if (nbFiles == 0) {
       // no file needs to be splitted.
-      return new Pair<Integer, Integer>(0, 0);
+      return new Pair<>(Collections.emptyList(), Collections.emptyList());
     }
     // Max #threads is the smaller of the number of storefiles or the default max determined above.
     int maxThreads = Math.min(
@@ -752,14 +756,18 @@ public class SplitTableRegionProcedure
       throw (InterruptedIOException) new InterruptedIOException().initCause(e);
     }
 
-    int daughterA = 0;
-    int daughterB = 0;
+    List<Path> daughterA = new ArrayList<>();
+    List<Path> daughterB = new ArrayList<>();
     // Look for any exception
     for (Future<Pair<Path, Path>> future : futures) {
       try {
         Pair<Path, Path> p = future.get();
-        daughterA += p.getFirst() != null ? 1 : 0;
-        daughterB += p.getSecond() != null ? 1 : 0;
+        if(p.getFirst() != null){
+          daughterA.add(p.getFirst());
+        }
+        if(p.getSecond() != null){
+          daughterB.add(p.getSecond());
+        }
       } catch (InterruptedException e) {
         throw (InterruptedIOException) new InterruptedIOException().initCause(e);
       } catch (ExecutionException e) {
@@ -772,7 +780,7 @@ public class SplitTableRegionProcedure
           getParentRegion().getShortNameToLog() + " Daughter A: " + daughterA +
           " storefiles, Daughter B: " + daughterB + " storefiles.");
     }
-    return new Pair<Integer, Integer>(daughterA, daughterB);
+    return new Pair<>(daughterA, daughterB);
   }
 
   private void assertSplitResultFilesCount(final FileSystem fs,
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 2f5f8d7..cb30432 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
@@ -24,7 +24,9 @@ import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.Objects;
 import java.util.Optional;
 import java.util.UUID;
@@ -49,6 +51,9 @@ import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.fs.HFileSystem;
 import org.apache.hadoop.hbase.io.HFileLink;
 import org.apache.hadoop.hbase.io.Reference;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTracker;
+import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.FSUtils;
@@ -595,19 +600,46 @@ public class HRegionFileSystem {
    * @param regionInfo daughter {@link org.apache.hadoop.hbase.client.RegionInfo}
    * @throws IOException
    */
-  public Path commitDaughterRegion(final RegionInfo regionInfo)
-      throws IOException {
+  public Path commitDaughterRegion(final RegionInfo regionInfo, List<Path> allRegionFiles,
+      MasterProcedureEnv env) throws IOException {
     Path regionDir = this.getSplitsDir(regionInfo);
     if (fs.exists(regionDir)) {
       // Write HRI to a file in case we need to recover hbase:meta
       Path regionInfoFile = new Path(regionDir, REGION_INFO_FILE);
       byte[] regionInfoContent = getRegionInfoFileContent(regionInfo);
       writeRegionInfoFileContent(conf, fs, regionInfoFile, regionInfoContent);
+      HRegionFileSystem regionFs = HRegionFileSystem.openRegionFromFileSystem(
+        env.getMasterConfiguration(), fs, getTableDir(), regionInfo, false);
+      insertRegionFilesIntoStoreTracker(allRegionFiles, env, regionFs);
     }
-
     return regionDir;
   }
 
+  private void insertRegionFilesIntoStoreTracker(List<Path> allFiles, MasterProcedureEnv env,
+      HRegionFileSystem regionFs) throws IOException {
+    TableDescriptor tblDesc = env.getMasterServices().getTableDescriptors().
+      get(regionInfo.getTable());
+    //we need to map trackers per store
+    Map<String, StoreFileTracker> trackerMap = new HashMap<>();
+    //we need to map store files per store
+    Map<String, List<StoreFileInfo>> fileInfoMap = new HashMap<>();
+    for(Path file : allFiles) {
+      String familyName = file.getParent().getName();
+      trackerMap.computeIfAbsent(familyName, t -> {
+        Configuration config = StoreFileTrackerFactory.mergeConfigurations(conf, tblDesc,
+          tblDesc.getColumnFamily(Bytes.toBytes(familyName)));
+        return StoreFileTrackerFactory.
+          create(config, true, familyName, regionFs);
+      });
+      fileInfoMap.computeIfAbsent(familyName, l -> new ArrayList<>());
+      List<StoreFileInfo> infos = fileInfoMap.get(familyName);
+      infos.add(new StoreFileInfo(conf, fs, file, true));
+    }
+    for(Map.Entry<String, StoreFileTracker> entry : trackerMap.entrySet()) {
+      entry.getValue().add(fileInfoMap.get(entry.getKey()));
+    }
+  }
+
   /**
    * Creates region split daughter directories under the table dir. If the daughter regions already
    * exist, for example, in the case of a recovery from a previous failed split procedure, this
@@ -795,13 +827,15 @@ public class HRegionFileSystem {
    * Commit a merged region, making it ready for use.
    * @throws IOException
    */
-  public void commitMergedRegion() throws IOException {
+  public void commitMergedRegion(List<Path> allMergedFiles, MasterProcedureEnv env)
+      throws IOException {
     Path regionDir = getMergesDir(regionInfoForFs);
     if (regionDir != null && fs.exists(regionDir)) {
       // Write HRI to a file in case we need to recover hbase:meta
       Path regionInfoFile = new Path(regionDir, REGION_INFO_FILE);
       byte[] regionInfoContent = getRegionInfoFileContent(regionInfo);
       writeRegionInfoFileContent(conf, fs, regionInfoFile, regionInfoContent);
+      insertRegionFilesIntoStoreTracker(allMergedFiles, env, this);
     }
   }
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/DefaultStoreFileTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/DefaultStoreFileTracker.java
index fa04481..22e0513 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/DefaultStoreFileTracker.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/DefaultStoreFileTracker.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.util.Collection;
 import java.util.List;
 import org.apache.hadoop.conf.Configuration;
+
 import org.apache.hadoop.hbase.regionserver.StoreContext;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -32,8 +33,7 @@ import org.apache.yetus.audience.InterfaceAudience;
 @InterfaceAudience.Private
 class DefaultStoreFileTracker extends StoreFileTrackerBase {
 
-  public DefaultStoreFileTracker(Configuration conf, boolean isPrimaryReplica,
-    StoreContext ctx) {
+  public DefaultStoreFileTracker(Configuration conf, boolean isPrimaryReplica, StoreContext ctx) {
     super(conf, isPrimaryReplica, ctx);
   }
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java
index aadedc8..0a85abb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java
@@ -48,7 +48,6 @@ import org.apache.yetus.audience.InterfaceAudience;
  */
 @InterfaceAudience.Private
 public interface StoreFileTracker {
-
   /**
    * Load the store files list when opening a region.
    */
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java
index 6cdfaf4..c446d5a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java
@@ -18,22 +18,51 @@
 package org.apache.hadoop.hbase.regionserver.storefiletracker;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.CompoundConfiguration;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
 import org.apache.hadoop.hbase.regionserver.StoreContext;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ReflectionUtils;
 import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Factory method for creating store file tracker.
  */
 @InterfaceAudience.Private
 public final class StoreFileTrackerFactory {
-
   public static final String TRACK_IMPL = "hbase.store.file-tracker.impl";
+  private static final Logger LOG = LoggerFactory.getLogger(StoreFileTrackerFactory.class);
 
   public static StoreFileTracker create(Configuration conf, boolean isPrimaryReplica,
-    StoreContext ctx) {
+      StoreContext ctx) {
     Class<? extends StoreFileTracker> tracker =
       conf.getClass(TRACK_IMPL, DefaultStoreFileTracker.class, StoreFileTracker.class);
+    LOG.info("instantiating StoreFileTracker impl {}", tracker.getName());
     return ReflectionUtils.newInstance(tracker, conf, isPrimaryReplica, ctx);
   }
+
+  public static StoreFileTracker create(Configuration conf, boolean isPrimaryReplica, String family,
+      HRegionFileSystem regionFs) {
+    ColumnFamilyDescriptorBuilder fDescBuilder =
+      ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(family));
+    StoreContext ctx = StoreContext.getBuilder().
+      withColumnFamilyDescriptor(fDescBuilder.build()).
+      withRegionFileSystem(regionFs).
+      build();
+    return StoreFileTrackerFactory.create(conf, isPrimaryReplica, ctx);
+  }
+
+  public static Configuration mergeConfigurations(Configuration global,
+    TableDescriptor table, ColumnFamilyDescriptor family) {
+    return new CompoundConfiguration()
+      .add(global)
+      .addBytesMap(table.getValues())
+      .addStringMap(family.getConfiguration())
+      .addBytesMap(family.getValues());
+  }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultStoreEngine.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultStoreEngine.java
index 3784876..523f277 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultStoreEngine.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultStoreEngine.java
@@ -67,6 +67,7 @@ public class TestDefaultStoreEngine {
         DummyStoreFlusher.class.getName());
     HRegion mockRegion = Mockito.mock(HRegion.class);
     HStore mockStore = Mockito.mock(HStore.class);
+    mockStore.conf = conf;
     Mockito.when(mockStore.getRegionInfo()).thenReturn(RegionInfoBuilder.FIRST_META_REGIONINFO);
     Mockito.when(mockStore.getHRegion()).thenReturn(mockRegion);
     StoreEngine<?, ?, ?, ?> se =
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDirectStoreSplitsMerges.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDirectStoreSplitsMerges.java
index 888ab53..87049d7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDirectStoreSplitsMerges.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDirectStoreSplitsMerges.java
@@ -21,6 +21,7 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.hadoop.fs.Path;
@@ -32,6 +33,7 @@ import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.master.assignment.SplitTableRegionProcedure;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
@@ -139,7 +141,9 @@ public class TestDirectStoreSplitsMerges {
         setRegionId(region.getRegionInfo().getRegionId() +
           EnvironmentEdgeManager.currentTime()).build();
     Path splitDir = regionFS.getSplitsDir(daughterA);
-    Path result = regionFS.commitDaughterRegion(daughterA);
+    MasterProcedureEnv env = TEST_UTIL.getMiniHBaseCluster().getMaster().
+      getMasterProcedureExecutor().getEnvironment();
+    Path result = regionFS.commitDaughterRegion(daughterA, new ArrayList<>(), env);
     assertEquals(splitDir, result);
   }
 
@@ -162,14 +166,18 @@ public class TestDirectStoreSplitsMerges {
     Path splitDirA = regionFS.getSplitsDir(daughterA);
     Path splitDirB = regionFS.getSplitsDir(daughterB);
     HStoreFile file = (HStoreFile) region.getStore(FAMILY_NAME).getStorefiles().toArray()[0];
-    regionFS
+    List<Path> filesA = new ArrayList<>();
+    filesA.add(regionFS
       .splitStoreFile(daughterA, Bytes.toString(FAMILY_NAME), file,
-        Bytes.toBytes("002"), false, region.getSplitPolicy());
-    regionFS
+        Bytes.toBytes("002"), false, region.getSplitPolicy()));
+    List<Path> filesB = new ArrayList<>();
+    filesB.add(regionFS
       .splitStoreFile(daughterB, Bytes.toString(FAMILY_NAME), file,
-        Bytes.toBytes("002"), true, region.getSplitPolicy());
-    Path resultA = regionFS.commitDaughterRegion(daughterA);
-    Path resultB = regionFS.commitDaughterRegion(daughterB);
+        Bytes.toBytes("002"), true, region.getSplitPolicy()));
+    MasterProcedureEnv env = TEST_UTIL.getMiniHBaseCluster().getMaster().
+      getMasterProcedureExecutor().getEnvironment();
+    Path resultA = regionFS.commitDaughterRegion(daughterA, filesA, env);
+    Path resultB = regionFS.commitDaughterRegion(daughterB, filesB, env);
     assertEquals(splitDirA, resultA);
     assertEquals(splitDirB, resultB);
   }
@@ -203,8 +211,11 @@ public class TestDirectStoreSplitsMerges {
     mergeFileFromRegion(mergeRegionFs, first, file);
     //merge file from second region
     file = (HStoreFile) second.getStore(FAMILY_NAME).getStorefiles().toArray()[0];
-    mergeFileFromRegion(mergeRegionFs, second, file);
-    mergeRegionFs.commitMergedRegion();
+    List<Path> mergedFiles = new ArrayList<>();
+    mergedFiles.add(mergeFileFromRegion(mergeRegionFs, second, file));
+    MasterProcedureEnv env = TEST_UTIL.getMiniHBaseCluster().getMaster().
+      getMasterProcedureExecutor().getEnvironment();
+    mergeRegionFs.commitMergedRegion(mergedFiles, env);
   }
 
   private void waitForSplitProcComplete(int attempts, int waitTime) throws Exception {
@@ -223,11 +234,12 @@ public class TestDirectStoreSplitsMerges {
     }
   }
 
-  private void mergeFileFromRegion(HRegionFileSystem regionFS, HRegion regionToMerge,
+  private Path mergeFileFromRegion(HRegionFileSystem regionFS, HRegion regionToMerge,
       HStoreFile file) throws IOException {
     Path mergedFile = regionFS.mergeStoreFile(regionToMerge.getRegionInfo(),
       Bytes.toString(FAMILY_NAME), file);
     validateResultingFile(regionToMerge.getRegionInfo().getEncodedName(), mergedFile);
+    return mergedFile;
   }
 
   private void validateResultingFile(String originalRegion, Path result){
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 de473d0..efe6fce 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
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
+import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -44,12 +45,14 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.PrivateCellUtil;
+import org.apache.hadoop.hbase.TableDescriptors;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
 import org.apache.hadoop.hbase.io.HFileLink;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
@@ -65,6 +68,8 @@ import org.apache.hadoop.hbase.io.hfile.HFileInfo;
 import org.apache.hadoop.hbase.io.hfile.HFileScanner;
 import org.apache.hadoop.hbase.io.hfile.ReaderContext;
 import org.apache.hadoop.hbase.io.hfile.ReaderContextBuilder;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.BloomFilterFactory;
@@ -1090,7 +1095,19 @@ public class TestHStoreFile extends HBaseTestCase {
     if (null == path) {
       return null;
     }
-    Path regionDir = regionFs.commitDaughterRegion(hri);
+    List<Path> splitFiles = new ArrayList<>();
+    splitFiles.add(path);
+    MasterProcedureEnv mockEnv = mock(MasterProcedureEnv.class);
+    MasterServices mockServices = mock(MasterServices.class);
+    when(mockEnv.getMasterServices()).thenReturn(mockServices);
+    when(mockEnv.getMasterConfiguration()).thenReturn(new Configuration());
+    TableDescriptors mockTblDescs = mock(TableDescriptors.class);
+    when(mockServices.getTableDescriptors()).thenReturn(mockTblDescs);
+    TableDescriptor mockTblDesc = mock(TableDescriptor.class);
+    when(mockTblDescs.get(any())).thenReturn(mockTblDesc);
+    ColumnFamilyDescriptor mockCfDesc = mock(ColumnFamilyDescriptor.class);
+    when(mockTblDesc.getColumnFamily(any())).thenReturn(mockCfDesc);
+    Path regionDir = regionFs.commitDaughterRegion(hri, splitFiles, mockEnv);
     return new Path(new Path(regionDir, family), path.getName());
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMergesSplitsAddToTracker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMergesSplitsAddToTracker.java
new file mode 100644
index 0000000..6a9e08f
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMergesSplitsAddToTracker.java
@@ -0,0 +1,267 @@
+/**
+ * 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.regionserver;
+
+import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.
+  TRACK_IMPL;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import org.apache.commons.lang3.mutable.MutableBoolean;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.regionserver.storefiletracker.TestStoreFileTracker;
+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.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.Pair;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+
+@Category({RegionServerTests.class, LargeTests.class})
+public class TestMergesSplitsAddToTracker {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestMergesSplitsAddToTracker.class);
+
+  private static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+
+  public static final byte[] FAMILY_NAME = Bytes.toBytes("info");
+
+  @Rule
+  public TestName name = new TestName();
+
+  @BeforeClass
+  public static void setupClass() throws Exception {
+    TEST_UTIL.getConfiguration().set(TRACK_IMPL, TestStoreFileTracker.class.getName());
+    TEST_UTIL.startMiniCluster();
+  }
+
+  @AfterClass
+  public static void afterClass() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  @Before
+  public void setup(){
+    TestStoreFileTracker.trackedFiles = new HashMap<>();
+  }
+
+  @Test
+  public void testCommitDaughterRegion() throws Exception {
+    TableName table = TableName.valueOf(name.getMethodName());
+    TEST_UTIL.createTable(table, FAMILY_NAME);
+    //first put some data in order to have a store file created
+    putThreeRowsAndFlush(table);
+    HRegion region = TEST_UTIL.getHBaseCluster().getRegions(table).get(0);
+    HRegionFileSystem regionFS = region.getStores().get(0).getRegionFileSystem();
+    RegionInfo daughterA =
+      RegionInfoBuilder.newBuilder(table).setStartKey(region.getRegionInfo().getStartKey()).
+        setEndKey(Bytes.toBytes("002")).setSplit(false).
+        setRegionId(region.getRegionInfo().getRegionId() +
+          EnvironmentEdgeManager.currentTime()).
+        build();
+    RegionInfo daughterB = RegionInfoBuilder.newBuilder(table).setStartKey(Bytes.toBytes("002"))
+      .setEndKey(region.getRegionInfo().getEndKey()).setSplit(false)
+      .setRegionId(region.getRegionInfo().getRegionId()).build();
+    HStoreFile file = (HStoreFile) region.getStore(FAMILY_NAME).getStorefiles().toArray()[0];
+    List<Path> splitFilesA = new ArrayList<>();
+    splitFilesA.add(regionFS
+      .splitStoreFile(daughterA, Bytes.toString(FAMILY_NAME), file,
+        Bytes.toBytes("002"), false, region.getSplitPolicy()));
+    List<Path> splitFilesB = new ArrayList<>();
+    splitFilesB.add(regionFS
+      .splitStoreFile(daughterB, Bytes.toString(FAMILY_NAME), file,
+        Bytes.toBytes("002"), true, region.getSplitPolicy()));
+    MasterProcedureEnv env = TEST_UTIL.getMiniHBaseCluster().getMaster().
+      getMasterProcedureExecutor().getEnvironment();
+    Path resultA = regionFS.commitDaughterRegion(daughterA, splitFilesA, env);
+    Path resultB = regionFS.commitDaughterRegion(daughterB, splitFilesB, env);
+    FileSystem fs = regionFS.getFileSystem();
+    verifyFilesAreTracked(resultA, fs);
+    verifyFilesAreTracked(resultB, fs);
+  }
+
+  @Test
+  public void testCommitMergedRegion() throws Exception {
+    TableName table = TableName.valueOf(name.getMethodName());
+    TEST_UTIL.createTable(table, FAMILY_NAME);
+    //splitting the table first
+    split(table, Bytes.toBytes("002"));
+    //Add data and flush to create files in the two different regions
+    putThreeRowsAndFlush(table);
+    List<HRegion> regions = TEST_UTIL.getHBaseCluster().getRegions(table);
+    HRegion first = regions.get(0);
+    HRegion second = regions.get(1);
+    HRegionFileSystem regionFS = first.getRegionFileSystem();
+
+    RegionInfo mergeResult =
+      RegionInfoBuilder.newBuilder(table).setStartKey(first.getRegionInfo().getStartKey())
+        .setEndKey(second.getRegionInfo().getEndKey()).setSplit(false)
+        .setRegionId(first.getRegionInfo().getRegionId() +
+          EnvironmentEdgeManager.currentTime()).build();
+
+    HRegionFileSystem mergeFS = HRegionFileSystem.createRegionOnFileSystem(
+      TEST_UTIL.getHBaseCluster().getMaster().getConfiguration(),
+      regionFS.getFileSystem(), regionFS.getTableDir(), mergeResult);
+
+    List<Path> mergedFiles = new ArrayList<>();
+    //merge file from first region
+    mergedFiles.add(mergeFileFromRegion(first, mergeFS));
+    //merge file from second region
+    mergedFiles.add(mergeFileFromRegion(second, mergeFS));
+    MasterProcedureEnv env = TEST_UTIL.getMiniHBaseCluster().getMaster().
+      getMasterProcedureExecutor().getEnvironment();
+    mergeFS.commitMergedRegion(mergedFiles, env);
+    //validate
+    FileSystem fs = first.getRegionFileSystem().getFileSystem();
+    Path finalMergeDir = new Path(first.getRegionFileSystem().getTableDir(),
+      mergeResult.getEncodedName());
+    verifyFilesAreTracked(finalMergeDir, fs);
+  }
+
+  @Test
+  public void testSplitLoadsFromTracker() throws Exception {
+    TableName table = TableName.valueOf(name.getMethodName());
+    TEST_UTIL.createTable(table, FAMILY_NAME);
+    //Add data and flush to create files in the two different regions
+    putThreeRowsAndFlush(table);
+    HRegion region = TEST_UTIL.getHBaseCluster().getRegions(table).get(0);
+    Pair<StoreFileInfo, String> copyResult = copyFileInTheStoreDir(region);
+    StoreFileInfo fileInfo = copyResult.getFirst();
+    String copyName = copyResult.getSecond();
+    //Now splits the region
+    split(table, Bytes.toBytes("002"));
+    List<HRegion> regions = TEST_UTIL.getHBaseCluster().getRegions(table);
+    HRegion first = regions.get(0);
+    validateDaughterRegionsFiles(first, fileInfo.getActiveFileName(), copyName);
+    HRegion second = regions.get(1);
+    validateDaughterRegionsFiles(second, fileInfo.getActiveFileName(), copyName);
+  }
+
+  private void split(TableName table, byte[] splitKey) throws IOException {
+    TEST_UTIL.getAdmin().split(table, splitKey);
+    // wait until split is done
+    TEST_UTIL.waitFor(30000, () -> TEST_UTIL.getHBaseCluster().getRegions(table).size() == 2);
+  }
+
+  @Test
+  public void testMergeLoadsFromTracker() throws Exception {
+    TableName table = TableName.valueOf(name.getMethodName());
+    TEST_UTIL.createTable(table, new byte[][]{FAMILY_NAME},
+      new byte[][]{Bytes.toBytes("002")});
+    //Add data and flush to create files in the two different regions
+    putThreeRowsAndFlush(table);
+    List<HRegion> regions = TEST_UTIL.getHBaseCluster().getRegions(table);
+    HRegion first = regions.get(0);
+    Pair<StoreFileInfo, String> copyResult = copyFileInTheStoreDir(first);
+    StoreFileInfo fileInfo = copyResult.getFirst();
+    String copyName = copyResult.getSecond();
+    //Now merges the first two regions
+    TEST_UTIL.getAdmin().mergeRegionsAsync(new byte[][]{
+      first.getRegionInfo().getEncodedNameAsBytes(),
+      regions.get(1).getRegionInfo().getEncodedNameAsBytes()
+    }, true).get(10, TimeUnit.SECONDS);
+    regions = TEST_UTIL.getHBaseCluster().getRegions(table);
+    HRegion merged = regions.get(0);
+    validateDaughterRegionsFiles(merged, fileInfo.getActiveFileName(), copyName);
+  }
+
+  private Pair<StoreFileInfo,String> copyFileInTheStoreDir(HRegion region) throws IOException {
+    Path storeDir = region.getRegionFileSystem().getStoreDir("info");
+    //gets the single file
+    StoreFileInfo fileInfo = region.getRegionFileSystem().getStoreFiles("info").get(0);
+    //make a copy of the valid file staight into the store dir, so that it's not tracked.
+    String copyName = UUID.randomUUID().toString().replaceAll("-", "");
+    Path copy = new Path(storeDir, copyName);
+    FileUtil.copy(region.getFilesystem(), fileInfo.getFileStatus(), region.getFilesystem(),
+      copy , false, false, TEST_UTIL.getConfiguration());
+    return new Pair<>(fileInfo, copyName);
+  }
+
+  private void validateDaughterRegionsFiles(HRegion region, String orignalFileName,
+      String untrackedFile) throws IOException {
+    //verify there's no link for the untracked, copied file in first region
+    List<StoreFileInfo> infos = region.getRegionFileSystem().getStoreFiles("info");
+    final MutableBoolean foundLink = new MutableBoolean(false);
+    infos.stream().forEach(i -> {
+      i.getActiveFileName().contains(orignalFileName);
+      if(i.getActiveFileName().contains(untrackedFile)){
+        fail();
+      }
+      if(i.getActiveFileName().contains(orignalFileName)){
+        foundLink.setTrue();
+      }
+    });
+    assertTrue(foundLink.booleanValue());
+  }
+
+  private void verifyFilesAreTracked(Path regionDir, FileSystem fs) throws Exception {
+    String storeId = regionDir.getName() + "-info";
+    for(FileStatus f : fs.listStatus(new Path(regionDir, Bytes.toString(FAMILY_NAME)))){
+      assertTrue(TestStoreFileTracker.trackedFiles.get(storeId).stream().filter(s ->
+        s.getPath().equals(f.getPath())).findFirst().isPresent());
+    }
+  }
+
+  private Path mergeFileFromRegion(HRegion regionToMerge, HRegionFileSystem mergeFS)
+      throws IOException {
+    HStoreFile file = (HStoreFile) regionToMerge.getStore(FAMILY_NAME).getStorefiles().toArray()[0];
+    return mergeFS.mergeStoreFile(regionToMerge.getRegionInfo(), Bytes.toString(FAMILY_NAME), file);
+  }
+
+  private void putThreeRowsAndFlush(TableName table) throws IOException {
+    Table tbl = TEST_UTIL.getConnection().getTable(table);
+    Put put = new Put(Bytes.toBytes("001"));
+    byte[] qualifier = Bytes.toBytes("1");
+    put.addColumn(FAMILY_NAME, qualifier, Bytes.toBytes(1));
+    tbl.put(put);
+    put = new Put(Bytes.toBytes("002"));
+    put.addColumn(FAMILY_NAME, qualifier, Bytes.toBytes(2));
+    tbl.put(put);
+    put = new Put(Bytes.toBytes("003"));
+    put.addColumn(FAMILY_NAME, qualifier, Bytes.toBytes(2));
+    tbl.put(put);
+    TEST_UTIL.flush(table);
+  }
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreEngine.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreEngine.java
index 498068b..f28ea1b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreEngine.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreEngine.java
@@ -120,6 +120,7 @@ public class TestStripeStoreEngine {
   private static TestStoreEngine createEngine(Configuration conf) throws Exception {
     HRegion region = mock(HRegion.class);
     HStore store = mock(HStore.class);
+    store.conf = conf;
     when(store.getRegionInfo()).thenReturn(RegionInfoBuilder.FIRST_META_REGIONINFO);
     when(store.getHRegion()).thenReturn(region);
     CellComparatorImpl kvComparator = mock(CellComparatorImpl.class);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/DefaultStoreFileTracker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTracker.java
similarity index 61%
copy from hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/DefaultStoreFileTracker.java
copy to hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTracker.java
index fa04481..05ca1fc 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/DefaultStoreFileTracker.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTracker.java
@@ -18,43 +18,39 @@
 package org.apache.hadoop.hbase.regionserver.storefiletracker;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Collection;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.regionserver.StoreContext;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
-import org.apache.yetus.audience.InterfaceAudience;
-
-/**
- * The default implementation for store file tracker, where we do not persist the store file list,
- * and use listing when loading store files.
- */
-@InterfaceAudience.Private
-class DefaultStoreFileTracker extends StoreFileTrackerBase {
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
-  public DefaultStoreFileTracker(Configuration conf, boolean isPrimaryReplica,
-    StoreContext ctx) {
-    super(conf, isPrimaryReplica, ctx);
-  }
+public class TestStoreFileTracker extends DefaultStoreFileTracker {
 
-  @Override
-  public List<StoreFileInfo> load() throws IOException {
-    return ctx.getRegionFileSystem().getStoreFiles(ctx.getFamily().getNameAsString());
-  }
+  private static final Logger LOG = LoggerFactory.getLogger(TestStoreFileTracker.class);
+  public static Map<String, List<StoreFileInfo>> trackedFiles = new HashMap<>();
+  private String storeId;
 
-  @Override
-  public boolean requireWritingToTmpDirFirst() {
-    return true;
+  public TestStoreFileTracker(Configuration conf, boolean isPrimaryReplica, StoreContext ctx) {
+    super(conf, isPrimaryReplica, ctx);
+    this.storeId = ctx.getRegionInfo().getEncodedName() + "-" + ctx.getFamily().getNameAsString();
+    LOG.info("created storeId: {}", storeId);
+    trackedFiles.computeIfAbsent(storeId, v -> new ArrayList<>());
   }
 
   @Override
   protected void doAddNewStoreFiles(Collection<StoreFileInfo> newFiles) throws IOException {
-    // NOOP
+    LOG.info("adding to storeId: {}", storeId);
+    trackedFiles.get(storeId).addAll(newFiles);
   }
 
   @Override
-  protected void doAddCompactionResults(Collection<StoreFileInfo> compactedFiles,
-    Collection<StoreFileInfo> newFiles) throws IOException {
-    // NOOP
+  public List<StoreFileInfo> load() throws IOException {
+    return trackedFiles.get(storeId);
   }
 }

[hbase] 13/15: HBASE-26271 Cleanup the broken store files under data directory (#3786)

Posted by zh...@apache.org.
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

commit 68252e17cb92a3605c20f428898f5bcf849bef57
Author: BukrosSzabolcs <sz...@cloudera.com>
AuthorDate: Tue Nov 9 17:19:00 2021 +0100

    HBASE-26271 Cleanup the broken store files under data directory (#3786)
    
    Signed-off-by: Duo Zhang <zh...@apache.org>
    Signed-off-by: Josh Elser <el...@apache.org>
    Signed-off-by: Wellington Ramos Chevreuil <wc...@apache.org>
---
 .../hadoop/hbase/mob/DefaultMobStoreCompactor.java |  16 +-
 .../regionserver/AbstractMultiFileWriter.java      |   6 +-
 .../hbase/regionserver/BrokenStoreFileCleaner.java | 202 ++++++++++++++++++
 .../regionserver/DateTieredMultiFileWriter.java    |   2 +-
 .../hbase/regionserver/HRegionFileSystem.java      |   2 +-
 .../hadoop/hbase/regionserver/HRegionServer.java   |  26 +++
 .../apache/hadoop/hbase/regionserver/HStore.java   |   6 +
 .../hadoop/hbase/regionserver/StoreEngine.java     |  21 ++
 .../hbase/regionserver/StripeMultiFileWriter.java  |   2 +-
 .../compactions/AbstractMultiOutputCompactor.java  |   4 +-
 .../hbase/regionserver/compactions/Compactor.java  |  45 +++-
 .../compactions/DateTieredCompactor.java           |   6 +-
 .../regionserver/compactions/DefaultCompactor.java |   9 +-
 .../regionserver/compactions/StripeCompactor.java  |   2 +-
 .../FileBasedStoreFileTracker.java                 |   2 +-
 .../MigrationStoreFileTracker.java                 |   2 +-
 .../storefiletracker/StoreFileTracker.java         |   6 +
 .../storefiletracker/StoreFileTrackerBase.java     |   6 -
 .../hbase/snapshot/RestoreSnapshotHelper.java      |   2 +-
 .../regionserver/TestBrokenStoreFileCleaner.java   | 226 +++++++++++++++++++++
 .../hbase/regionserver/TestCompactorMemLeak.java   |   4 +-
 .../storefiletracker/TestStoreFileTracker.java     |   1 -
 22 files changed, 566 insertions(+), 32 deletions(-)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java
index 1b218b4..dd800f9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java
@@ -28,7 +28,6 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.PrivateCellUtil;
-import org.apache.hadoop.hbase.regionserver.CellSink;
 import org.apache.hadoop.hbase.regionserver.HMobStore;
 import org.apache.hadoop.hbase.regionserver.HStore;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
@@ -52,6 +51,8 @@ import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
+
 /**
  * Compact passed set of files in the mob-enabled column family.
  */
@@ -154,7 +155,6 @@ public class DefaultMobStoreCompactor extends DefaultCompactor {
    * the scanner to filter the deleted cells.
    * @param fd File details
    * @param scanner Where to read from.
-   * @param writer Where to write to.
    * @param smallestReadPoint Smallest read point.
    * @param cleanSeqId When true, remove seqId(used to be mvcc) value which is <= smallestReadPoint
    * @param throughputController The compaction throughput controller.
@@ -163,7 +163,7 @@ public class DefaultMobStoreCompactor extends DefaultCompactor {
    * @return Whether compaction ended; false if it was interrupted for any reason.
    */
   @Override
-  protected boolean performCompaction(FileDetails fd, InternalScanner scanner, CellSink writer,
+  protected boolean performCompaction(FileDetails fd, InternalScanner scanner,
       long smallestReadPoint, boolean cleanSeqId, ThroughputController throughputController,
       boolean major, int numofFilesToCompact) throws IOException {
     long bytesWrittenProgressForCloseCheck = 0;
@@ -369,4 +369,14 @@ public class DefaultMobStoreCompactor extends DefaultCompactor {
     progress.complete();
     return true;
   }
+
+
+  @Override
+  protected List<Path> commitWriter(FileDetails fd,
+      CompactionRequestImpl request) throws IOException {
+    List<Path> newFiles = Lists.newArrayList(writer.getPath());
+    writer.appendMetadata(fd.maxSeqId, request.isAllFiles(), request.getFiles());
+    writer.close();
+    return newFiles;
+  }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMultiFileWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMultiFileWriter.java
index f250304..82c3867 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMultiFileWriter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMultiFileWriter.java
@@ -110,7 +110,11 @@ public abstract class AbstractMultiFileWriter implements CellSink, ShipperListen
     return paths;
   }
 
-  protected abstract Collection<StoreFileWriter> writers();
+  /**
+   * Returns all writers. This is used to prevent deleting currently writen storefiles
+   * during cleanup.
+   */
+  public abstract Collection<StoreFileWriter> writers();
 
   /**
    * Subclasses override this method to be called at the end of a successful sequence of append; all
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BrokenStoreFileCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BrokenStoreFileCleaner.java
new file mode 100644
index 0000000..0c4807d
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BrokenStoreFileCleaner.java
@@ -0,0 +1,202 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.regionserver;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.io.HFileLink;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This Chore, every time it runs, will clear the unsused HFiles in the data
+ * folder.
+ */
+@InterfaceAudience.Private
+public class BrokenStoreFileCleaner extends ScheduledChore {
+  private static final Logger LOG = LoggerFactory.getLogger(BrokenStoreFileCleaner.class);
+  public static final String BROKEN_STOREFILE_CLEANER_ENABLED =
+      "hbase.region.broken.storefilecleaner.enabled";
+  public static final boolean DEFAULT_BROKEN_STOREFILE_CLEANER_ENABLED = false;
+  public static final String BROKEN_STOREFILE_CLEANER_TTL =
+      "hbase.region.broken.storefilecleaner.ttl";
+  public static final long DEFAULT_BROKEN_STOREFILE_CLEANER_TTL = 1000 * 60 * 60 * 12; //12h
+  public static final String BROKEN_STOREFILE_CLEANER_DELAY =
+      "hbase.region.broken.storefilecleaner.delay";
+  public static final int DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY = 1000 * 60 * 60 * 2; //2h
+  public static final String BROKEN_STOREFILE_CLEANER_DELAY_JITTER =
+      "hbase.region.broken.storefilecleaner.delay.jitter";
+  public static final double DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY_JITTER = 0.25D;
+  public static final String BROKEN_STOREFILE_CLEANER_PERIOD =
+      "hbase.region.broken.storefilecleaner.period";
+  public static final int DEFAULT_BROKEN_STOREFILE_CLEANER_PERIOD = 1000 * 60 * 60 * 6; //6h
+
+  private HRegionServer regionServer;
+  private final AtomicBoolean enabled = new AtomicBoolean(true);
+  private long fileTtl;
+
+  public BrokenStoreFileCleaner(final int delay, final int period, final Stoppable stopper,
+    Configuration conf, HRegionServer regionServer) {
+    super("BrokenStoreFileCleaner", stopper, period, delay);
+    this.regionServer = regionServer;
+    setEnabled(
+      conf.getBoolean(BROKEN_STOREFILE_CLEANER_ENABLED, DEFAULT_BROKEN_STOREFILE_CLEANER_ENABLED));
+    fileTtl = conf.getLong(BROKEN_STOREFILE_CLEANER_TTL, DEFAULT_BROKEN_STOREFILE_CLEANER_TTL);
+  }
+
+  public boolean setEnabled(final boolean enabled) {
+    return this.enabled.getAndSet(enabled);
+  }
+
+  public boolean getEnabled() {
+    return this.enabled.get();
+  }
+
+  @Override
+  public void chore() {
+    if (getEnabled()) {
+      long start = EnvironmentEdgeManager.currentTime();
+      AtomicLong deletedFiles = new AtomicLong(0);
+      AtomicLong failedDeletes = new AtomicLong(0);
+      for (HRegion region : regionServer.getRegions()) {
+        for (HStore store : region.getStores()) {
+          //only do cleanup in stores not using tmp directories
+          if (store.getStoreEngine().requireWritingToTmpDirFirst()) {
+            continue;
+          }
+          Path storePath =
+              new Path(region.getRegionFileSystem().getRegionDir(), store.getColumnFamilyName());
+
+          try {
+            List<FileStatus> fsStoreFiles =
+              Arrays.asList(region.getRegionFileSystem().fs.listStatus(storePath));
+            fsStoreFiles.forEach(
+              file -> cleanFileIfNeeded(file, store, deletedFiles, failedDeletes));
+          } catch (IOException e) {
+            LOG.warn("Failed to list files in {}, cleanup is skipped there",storePath);
+            continue;
+          }
+        }
+      }
+      LOG.debug(
+        "BrokenStoreFileCleaner on {} run for: {}ms. It deleted {} files and tried but failed "
+        + "to delete {}",
+        regionServer.getServerName().getServerName(), EnvironmentEdgeManager.currentTime() - start,
+        deletedFiles.get(), failedDeletes.get());
+    } else {
+      LOG.trace("Broken storefile Cleaner chore disabled! Not cleaning.");
+    }
+  }
+
+  private void cleanFileIfNeeded(FileStatus file, HStore store,
+    AtomicLong deletedFiles, AtomicLong failedDeletes) {
+    if(file.isDirectory()){
+      LOG.trace("This is a Directory {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(!validate(file.getPath())){
+      LOG.trace("Invalid file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(!isOldEnough(file)){
+      LOG.trace("Fresh file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isActiveStorefile(file, store)){
+      LOG.trace("Actively used storefile file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    // Compacted files can still have readers and are cleaned by a separate chore, so they have to
+    // be skipped here
+    if(isCompactedFile(file, store)){
+      LOG.trace("Cleanup is done by a different chore for file {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    if(isCompactionResultFile(file, store)){
+      LOG.trace("The file is the result of an ongoing compaction {}, skip cleanup", file.getPath());
+      return;
+    }
+
+    deleteFile(file, store, deletedFiles, failedDeletes);
+  }
+
+  private boolean isCompactionResultFile(FileStatus file, HStore store) {
+    return store.getStoreEngine().getCompactor().getCompactionTargets().contains(file.getPath());
+  }
+
+  // Compacted files can still have readers and are cleaned by a separate chore, so they have to
+  // be skipped here
+  private boolean isCompactedFile(FileStatus file, HStore store) {
+    return store.getStoreEngine().getStoreFileManager().getCompactedfiles().stream()
+      .anyMatch(sf -> sf.getPath().equals(file.getPath()));
+  }
+
+  private boolean isActiveStorefile(FileStatus file, HStore store) {
+    return store.getStoreEngine().getStoreFileManager().getStorefiles().stream()
+      .anyMatch(sf -> sf.getPath().equals(file.getPath()));
+  }
+
+  boolean validate(Path file) {
+    if (HFileLink.isBackReferencesDir(file) || HFileLink.isBackReferencesDir(file.getParent())) {
+      return true;
+    }
+    return StoreFileInfo.validateStoreFileName(file.getName());
+  }
+
+  boolean isOldEnough(FileStatus file){
+    return file.getModificationTime() + fileTtl < EnvironmentEdgeManager.currentTime();
+  }
+
+  private void deleteFile(FileStatus file, HStore store, AtomicLong deletedFiles,
+    AtomicLong failedDeletes) {
+    Path filePath = file.getPath();
+    LOG.debug("Removing {} from store", filePath);
+    try {
+      boolean success = store.getFileSystem().delete(filePath, false);
+      if (!success) {
+        failedDeletes.incrementAndGet();
+        LOG.warn("Attempted to delete:" + filePath
+            + ", but couldn't. Attempt to delete on next pass.");
+      }
+      else{
+        deletedFiles.incrementAndGet();
+      }
+    } catch (IOException e) {
+      e = e instanceof RemoteException ?
+          ((RemoteException)e).unwrapRemoteException() : e;
+      LOG.warn("Error while deleting: " + filePath, e);
+    }
+  }
+
+}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DateTieredMultiFileWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DateTieredMultiFileWriter.java
index 8201cb1..1e10eb2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DateTieredMultiFileWriter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DateTieredMultiFileWriter.java
@@ -71,7 +71,7 @@ public class DateTieredMultiFileWriter extends AbstractMultiFileWriter {
   }
 
   @Override
-  protected Collection<StoreFileWriter> writers() {
+  public Collection<StoreFileWriter> writers() {
     return lowerBoundary2Writer.values();
   }
 
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 8110025..8920471 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
@@ -609,7 +609,7 @@ public class HRegionFileSystem {
       writeRegionInfoFileContent(conf, fs, regionInfoFile, regionInfoContent);
       HRegionFileSystem regionFs = HRegionFileSystem.openRegionFromFileSystem(
         env.getMasterConfiguration(), fs, getTableDir(), regionInfo, false);
-        insertRegionFilesIntoStoreTracker(allRegionFiles, env, regionFs);
+      insertRegionFilesIntoStoreTracker(allRegionFiles, env, regionFs);
     }
     return regionDir;
   }
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 57ab15b..3066b50 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
@@ -546,6 +546,8 @@ public class HRegionServer extends Thread implements
    */
   protected final ConfigurationManager configurationManager;
 
+  private BrokenStoreFileCleaner brokenStoreFileCleaner;
+
   @InterfaceAudience.Private
   CompactedHFilesDischarger compactedFileDischarger;
 
@@ -2156,6 +2158,9 @@ public class HRegionServer extends Thread implements
     if (this.slowLogTableOpsChore != null) {
       choreService.scheduleChore(slowLogTableOpsChore);
     }
+    if (this.brokenStoreFileCleaner != null) {
+      choreService.scheduleChore(brokenStoreFileCleaner);
+    }
 
     // Leases is not a Thread. Internally it runs a daemon thread. If it gets
     // an unhandled exception, it will just exit.
@@ -2236,6 +2241,22 @@ public class HRegionServer extends Thread implements
       this.storefileRefresher = new StorefileRefresherChore(storefileRefreshPeriod,
           onlyMetaRefresh, this, this);
     }
+
+    int brokenStoreFileCleanerPeriod  = conf.getInt(
+      BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_PERIOD,
+      BrokenStoreFileCleaner.DEFAULT_BROKEN_STOREFILE_CLEANER_PERIOD);
+    int brokenStoreFileCleanerDelay  = conf.getInt(
+      BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_DELAY,
+      BrokenStoreFileCleaner.DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY);
+    double brokenStoreFileCleanerDelayJitter = conf.getDouble(
+      BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_DELAY_JITTER,
+      BrokenStoreFileCleaner.DEFAULT_BROKEN_STOREFILE_CLEANER_DELAY_JITTER);
+    double jitterRate = (RandomUtils.nextDouble() - 0.5D) * brokenStoreFileCleanerDelayJitter;
+    long jitterValue = Math.round(brokenStoreFileCleanerDelay * jitterRate);
+    this.brokenStoreFileCleaner =
+      new BrokenStoreFileCleaner((int) (brokenStoreFileCleanerDelay + jitterValue),
+        brokenStoreFileCleanerPeriod, this, conf, this);
+
     registerConfigurationObservers();
   }
 
@@ -4027,4 +4048,9 @@ public class HRegionServer extends Thread implements
   public MetaRegionLocationCache getMetaRegionLocationCache() {
     return this.metaRegionLocationCache;
   }
+
+  @InterfaceAudience.Private
+  public BrokenStoreFileCleaner getBrokenStoreFileCleaner(){
+    return brokenStoreFileCleaner;
+  }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
index 5309305..e910f3c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
@@ -1156,6 +1156,12 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
       }
     }
     replaceStoreFiles(filesToCompact, sfs, true);
+
+    // This step is necessary for the correctness of BrokenStoreFileCleanerChore. It lets the
+    // CleanerChore know that compaction is done and the file can be cleaned up if compaction
+    // have failed.
+    storeEngine.resetCompactionWriter();
+
     if (cr.isMajor()) {
       majorCompactedCellsCount.addAndGet(getCompactionProgress().getTotalCompactingKVs());
       majorCompactedCellsSize.addAndGet(getCompactionProgress().totalCompactedSize);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreEngine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreEngine.java
index 0486729..ddb52d1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreEngine.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreEngine.java
@@ -42,9 +42,11 @@ import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.log.HBaseMarkers;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionPolicy;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequestImpl;
 import org.apache.hadoop.hbase.regionserver.compactions.Compactor;
 import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTracker;
 import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory;
+import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.ReflectionUtils;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
@@ -532,6 +534,25 @@ public abstract class StoreEngine<SF extends StoreFlusher, CP extends Compaction
     }
   }
 
+  /**
+   * Whether the implementation of the used storefile tracker requires you to write to temp
+   * directory first, i.e, does not allow broken store files under the actual data directory.
+   */
+  public boolean requireWritingToTmpDirFirst() {
+    return storeFileTracker.requireWritingToTmpDirFirst();
+  }
+
+  /**
+   * Resets the compaction writer when the new file is committed and used as active storefile.
+   * This step is necessary for the correctness of BrokenStoreFileCleanerChore. It lets the
+   * CleanerChore know that compaction is done and the file can be cleaned up if compaction
+   * have failed. Currently called in
+   * @see HStore#doCompaction(CompactionRequestImpl, Collection, User, long, List)
+   */
+  public void resetCompactionWriter(){
+    compactor.resetWriter();
+  }
+
   @RestrictedApi(explanation = "Should only be called in TestHStore", link = "",
     allowedOnPath = ".*/TestHStore.java")
   ReadWriteLock getLock() {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeMultiFileWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeMultiFileWriter.java
index fc0598d..a4e943a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeMultiFileWriter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeMultiFileWriter.java
@@ -58,7 +58,7 @@ public abstract class StripeMultiFileWriter extends AbstractMultiFileWriter {
   }
 
   @Override
-  protected Collection<StoreFileWriter> writers() {
+  public Collection<StoreFileWriter> writers() {
     return existingWriters;
   }
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/AbstractMultiOutputCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/AbstractMultiOutputCompactor.java
index 533be17..19b7a98 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/AbstractMultiOutputCompactor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/AbstractMultiOutputCompactor.java
@@ -68,7 +68,7 @@ public abstract class AbstractMultiOutputCompactor<T extends AbstractMultiFileWr
   }
 
   @Override
-  protected void abortWriter(T writer) throws IOException {
+  protected void abortWriter() throws IOException {
     FileSystem fs = store.getFileSystem();
     for (Path leftoverFile : writer.abortWriters()) {
       try {
@@ -79,5 +79,7 @@ public abstract class AbstractMultiOutputCompactor<T extends AbstractMultiFileWr
           e);
       }
     }
+    //this step signals that the target file is no longer writen and can be cleaned up
+    writer = null;
   }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
index 8178fb1..a821a90 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
@@ -25,9 +25,12 @@ import static org.apache.hadoop.hbase.regionserver.ScanType.COMPACT_RETAIN_DELET
 import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
+import java.util.stream.Collectors;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
@@ -36,6 +39,7 @@ import org.apache.hadoop.hbase.PrivateCellUtil;
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.io.hfile.HFileInfo;
+import org.apache.hadoop.hbase.regionserver.AbstractMultiFileWriter;
 import org.apache.hadoop.hbase.regionserver.CellSink;
 import org.apache.hadoop.hbase.regionserver.CreateStoreFileWriterParams;
 import org.apache.hadoop.hbase.regionserver.HStore;
@@ -91,6 +95,8 @@ public abstract class Compactor<T extends CellSink> {
   private final boolean dropCacheMajor;
   private final boolean dropCacheMinor;
 
+  protected T writer = null;
+
   //TODO: depending on Store is not good but, realistically, all compactors currently do.
   Compactor(Configuration conf, HStore store) {
     this.conf = conf;
@@ -323,7 +329,6 @@ public abstract class Compactor<T extends CellSink> {
     // Find the smallest read point across all the Scanners.
     long smallestReadPoint = getSmallestReadPoint();
 
-    T writer = null;
     boolean dropCache;
     if (request.isMajor() || request.isAllFiles()) {
       dropCache = this.dropCacheMajor;
@@ -347,8 +352,13 @@ public abstract class Compactor<T extends CellSink> {
         smallestReadPoint = Math.min(fd.minSeqIdToKeep, smallestReadPoint);
         cleanSeqId = true;
       }
+      if (writer != null){
+        LOG.warn("Writer exists when it should not: " + getCompactionTargets().stream()
+          .map(n -> n.toString())
+          .collect(Collectors.joining(", ", "{ ", " }")));
+      }
       writer = sinkFactory.createWriter(scanner, fd, dropCache, request.isMajor());
-      finished = performCompaction(fd, scanner, writer, smallestReadPoint, cleanSeqId,
+      finished = performCompaction(fd, scanner, smallestReadPoint, cleanSeqId,
         throughputController, request.isAllFiles(), request.getFiles().size());
       if (!finished) {
         throw new InterruptedIOException("Aborting compaction of store " + store + " in region "
@@ -368,24 +378,23 @@ public abstract class Compactor<T extends CellSink> {
         Closeables.close(scanner, true);
       }
       if (!finished && writer != null) {
-        abortWriter(writer);
+        abortWriter();
       }
     }
     assert finished : "We should have exited the method on all error paths";
     assert writer != null : "Writer should be non-null if no error";
-    return commitWriter(writer, fd, request);
+    return commitWriter(fd, request);
   }
 
-  protected abstract List<Path> commitWriter(T writer, FileDetails fd,
+  protected abstract List<Path> commitWriter(FileDetails fd,
       CompactionRequestImpl request) throws IOException;
 
-  protected abstract void abortWriter(T writer) throws IOException;
+  protected abstract void abortWriter() throws IOException;
 
   /**
    * Performs the compaction.
    * @param fd FileDetails of cell sink writer
    * @param scanner Where to read from.
-   * @param writer Where to write to.
    * @param smallestReadPoint Smallest read point.
    * @param cleanSeqId When true, remove seqId(used to be mvcc) value which is &lt;=
    *          smallestReadPoint
@@ -393,7 +402,7 @@ public abstract class Compactor<T extends CellSink> {
    * @param numofFilesToCompact the number of files to compact
    * @return Whether compaction ended; false if it was interrupted for some reason.
    */
-  protected boolean performCompaction(FileDetails fd, InternalScanner scanner, CellSink writer,
+  protected boolean performCompaction(FileDetails fd, InternalScanner scanner,
       long smallestReadPoint, boolean cleanSeqId, ThroughputController throughputController,
       boolean major, int numofFilesToCompact) throws IOException {
     assert writer instanceof ShipperListener;
@@ -536,4 +545,24 @@ public abstract class Compactor<T extends CellSink> {
     return new StoreScanner(store, scanInfo, scanners, smallestReadPoint, earliestPutTs,
         dropDeletesFromRow, dropDeletesToRow);
   }
+
+  public List<Path> getCompactionTargets(){
+    if (writer == null){
+      return Collections.emptyList();
+    }
+    synchronized (writer){
+      if (writer instanceof StoreFileWriter){
+        return Arrays.asList(((StoreFileWriter)writer).getPath());
+      }
+      return ((AbstractMultiFileWriter)writer).writers().stream().map(sfw -> sfw.getPath()).collect(
+        Collectors.toList());
+    }
+  }
+
+  /**
+   * Reset the Writer when the new storefiles were successfully added
+   */
+  public void resetWriter(){
+    writer = null;
+  }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactor.java
index fd54330..43e037c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactor.java
@@ -79,8 +79,10 @@ public class DateTieredCompactor extends AbstractMultiOutputCompactor<DateTiered
   }
 
   @Override
-  protected List<Path> commitWriter(DateTieredMultiFileWriter writer, FileDetails fd,
+  protected List<Path> commitWriter(FileDetails fd,
       CompactionRequestImpl request) throws IOException {
-    return writer.commitWriters(fd.maxSeqId, request.isAllFiles(), request.getFiles());
+    List<Path> pathList =
+      writer.commitWriters(fd.maxSeqId, request.isAllFiles(), request.getFiles());
+    return pathList;
   }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java
index afa2429..ad2384a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java
@@ -63,7 +63,7 @@ public class DefaultCompactor extends Compactor<StoreFileWriter> {
   }
 
   @Override
-  protected List<Path> commitWriter(StoreFileWriter writer, FileDetails fd,
+  protected List<Path> commitWriter(FileDetails fd,
       CompactionRequestImpl request) throws IOException {
     List<Path> newFiles = Lists.newArrayList(writer.getPath());
     writer.appendMetadata(fd.maxSeqId, request.isAllFiles(), request.getFiles());
@@ -72,12 +72,19 @@ public class DefaultCompactor extends Compactor<StoreFileWriter> {
   }
 
   @Override
+  protected void abortWriter() throws IOException {
+    abortWriter(writer);
+  }
+
   protected void abortWriter(StoreFileWriter writer) throws IOException {
     Path leftoverFile = writer.getPath();
     try {
       writer.close();
     } catch (IOException e) {
       LOG.warn("Failed to close the writer after an unfinished compaction.", e);
+    } finally {
+      //this step signals that the target file is no longer writen and can be cleaned up
+      writer = null;
     }
     try {
       store.getFileSystem().delete(leftoverFile, false);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/StripeCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/StripeCompactor.java
index 547555e..060a11b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/StripeCompactor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/StripeCompactor.java
@@ -125,7 +125,7 @@ public class StripeCompactor extends AbstractMultiOutputCompactor<StripeMultiFil
   }
 
   @Override
-  protected List<Path> commitWriter(StripeMultiFileWriter writer, FileDetails fd,
+  protected List<Path> commitWriter(FileDetails fd,
       CompactionRequestImpl request) throws IOException {
     List<Path> newFiles = writer.commitWriters(fd.maxSeqId, request.isMajor(), request.getFiles());
     assert !newFiles.isEmpty() : "Should have produced an empty file to preserve metadata.";
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/FileBasedStoreFileTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/FileBasedStoreFileTracker.java
index 079b59b..8d9b66e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/FileBasedStoreFileTracker.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/FileBasedStoreFileTracker.java
@@ -95,7 +95,7 @@ class FileBasedStoreFileTracker extends StoreFileTrackerBase {
   }
 
   @Override
-  protected boolean requireWritingToTmpDirFirst() {
+  public boolean requireWritingToTmpDirFirst() {
     return false;
   }
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java
index a6648f2..53a474d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java
@@ -57,7 +57,7 @@ class MigrationStoreFileTracker extends StoreFileTrackerBase {
   }
 
   @Override
-  protected boolean requireWritingToTmpDirFirst() {
+  public boolean requireWritingToTmpDirFirst() {
     // Returns true if either of the two StoreFileTracker returns true.
     // For example, if we want to migrate from a tracker implementation which can ignore the broken
     // files under data directory to a tracker implementation which can not, if we still allow
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java
index f56a0dd..aabbe8d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java
@@ -88,4 +88,10 @@ public interface StoreFileTracker {
    * @param builder The table descriptor builder for the given table.
    */
   TableDescriptorBuilder updateWithTrackerConfigs(TableDescriptorBuilder builder);
+
+  /**
+   * Whether the implementation of this tracker requires you to write to temp directory first, i.e,
+   * does not allow broken store files under the actual data directory.
+   */
+  boolean requireWritingToTmpDirFirst();
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java
index b6de32b..db10f4d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java
@@ -173,12 +173,6 @@ abstract class StoreFileTrackerBase implements StoreFileTracker {
     return builder.build();
   }
 
-  /**
-   * Whether the implementation of this tracker requires you to write to temp directory first, i.e,
-   * does not allow broken store files under the actual data directory.
-   */
-  protected abstract boolean requireWritingToTmpDirFirst();
-
   protected abstract void doAddNewStoreFiles(Collection<StoreFileInfo> newFiles) throws IOException;
 
   protected abstract void doAddCompactionResults(Collection<StoreFileInfo> compactedFiles,
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 7c75e46..0f8a95f 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
@@ -549,7 +549,7 @@ public class RestoreSnapshotHelper {
                   " of snapshot=" + snapshotName+
                   " to region=" + regionInfo.getEncodedName() + " table=" + tableName);
           String fileName = restoreStoreFile(familyDir, regionInfo, storeFile, createBackRefs);
-          //mark the reference file to be added to tracker
+          // mark the reference file to be added to tracker
           filesToTrack.add(new StoreFileInfo(conf, fs,
             new Path(familyDir, fileName), true));
         }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBrokenStoreFileCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBrokenStoreFileCleaner.java
new file mode 100644
index 0000000..eb5e6c7
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBrokenStoreFileCleaner.java
@@ -0,0 +1,226 @@
+/**
+ * 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.regionserver;
+
+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.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ MediumTests.class, RegionServerTests.class })
+public class TestBrokenStoreFileCleaner {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestBrokenStoreFileCleaner.class);
+
+  private final HBaseTestingUtility testUtil = new HBaseTestingUtility();
+  private final static byte[] fam = Bytes.toBytes("cf_1");
+  private final static byte[] qual1 = Bytes.toBytes("qf_1");
+  private final static byte[] val = Bytes.toBytes("val");
+  private final static  String junkFileName = "409fad9a751c4e8c86d7f32581bdc156";
+  TableName tableName;
+
+
+  @Before
+  public void setUp() throws Exception {
+    testUtil.getConfiguration().set(StoreFileTrackerFactory.TRACKER_IMPL,
+      "org.apache.hadoop.hbase.regionserver.storefiletracker.FileBasedStoreFileTracker");
+    testUtil.getConfiguration()
+      .set(BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_ENABLED, "true");
+    testUtil.getConfiguration().set(BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_TTL, "0");
+    testUtil.getConfiguration()
+      .set(BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_PERIOD, "15000000");
+    testUtil.getConfiguration().set(BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_DELAY, "0");
+    testUtil.startMiniCluster(1);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    testUtil.deleteTable(tableName);
+    testUtil.shutdownMiniCluster();
+  }
+
+  @Test
+  public void testDeletingJunkFile() throws Exception {
+    tableName = TableName.valueOf(getClass().getSimpleName() + "testDeletingJunkFile");
+    createTableWithData(tableName);
+
+    HRegion region = testUtil.getMiniHBaseCluster().getRegions(tableName).get(0);
+    ServerName sn = testUtil.getMiniHBaseCluster()
+      .getServerHoldingRegion(tableName, region.getRegionInfo().getRegionName());
+    HRegionServer rs = testUtil.getMiniHBaseCluster().getRegionServer(sn);
+    BrokenStoreFileCleaner cleaner = rs.getBrokenStoreFileCleaner();
+
+    //create junk file
+    HStore store = region.getStore(fam);
+    Path cfPath = store.getRegionFileSystem().getStoreDir(store.getColumnFamilyName());
+    Path junkFilePath = new Path(cfPath, junkFileName);
+
+    FSDataOutputStream junkFileOS = store.getFileSystem().create(junkFilePath);
+    junkFileOS.writeUTF("hello");
+    junkFileOS.close();
+
+    int storeFiles =  store.getStorefilesCount();
+    assertTrue(storeFiles > 0);
+
+    //verify the file exist before the chore and missing afterwards
+    assertTrue(store.getFileSystem().exists(junkFilePath));
+    cleaner.chore();
+    assertFalse(store.getFileSystem().exists(junkFilePath));
+
+    //verify no storefile got deleted
+    int currentStoreFiles =  store.getStorefilesCount();
+    assertEquals(currentStoreFiles, storeFiles);
+
+  }
+
+  @Test
+  public void testSkippingCompactedFiles() throws Exception {
+    tableName = TableName.valueOf(getClass().getSimpleName() + "testSkippningCompactedFiles");
+    createTableWithData(tableName);
+
+    HRegion region = testUtil.getMiniHBaseCluster().getRegions(tableName).get(0);
+
+    ServerName sn = testUtil.getMiniHBaseCluster()
+      .getServerHoldingRegion(tableName, region.getRegionInfo().getRegionName());
+    HRegionServer rs = testUtil.getMiniHBaseCluster().getRegionServer(sn);
+    BrokenStoreFileCleaner cleaner = rs.getBrokenStoreFileCleaner();
+
+    //run major compaction to generate compaced files
+    region.compact(true);
+
+    //make sure there are compacted files
+    HStore store = region.getStore(fam);
+    int compactedFiles =  store.getCompactedFilesCount();
+    assertTrue(compactedFiles > 0);
+
+    cleaner.chore();
+
+    //verify none of the compacted files were deleted
+    int existingCompactedFiles =  store.getCompactedFilesCount();
+    assertEquals(compactedFiles, existingCompactedFiles);
+
+    //verify adding a junk file does not break anything
+    Path cfPath = store.getRegionFileSystem().getStoreDir(store.getColumnFamilyName());
+    Path junkFilePath = new Path(cfPath, junkFileName);
+
+    FSDataOutputStream junkFileOS = store.getFileSystem().create(junkFilePath);
+    junkFileOS.writeUTF("hello");
+    junkFileOS.close();
+
+    assertTrue(store.getFileSystem().exists(junkFilePath));
+    cleaner.setEnabled(true);
+    cleaner.chore();
+    assertFalse(store.getFileSystem().exists(junkFilePath));
+
+    //verify compacted files are still intact
+    existingCompactedFiles =  store.getCompactedFilesCount();
+    assertEquals(compactedFiles, existingCompactedFiles);
+  }
+
+  @Test
+  public void testJunkFileTTL() throws Exception {
+    tableName = TableName.valueOf(getClass().getSimpleName() + "testDeletingJunkFile");
+    createTableWithData(tableName);
+
+    HRegion region = testUtil.getMiniHBaseCluster().getRegions(tableName).get(0);
+    ServerName sn = testUtil.getMiniHBaseCluster()
+      .getServerHoldingRegion(tableName, region.getRegionInfo().getRegionName());
+    HRegionServer rs = testUtil.getMiniHBaseCluster().getRegionServer(sn);
+
+    //create junk file
+    HStore store = region.getStore(fam);
+    Path cfPath = store.getRegionFileSystem().getStoreDir(store.getColumnFamilyName());
+    Path junkFilePath = new Path(cfPath, junkFileName);
+
+    FSDataOutputStream junkFileOS = store.getFileSystem().create(junkFilePath);
+    junkFileOS.writeUTF("hello");
+    junkFileOS.close();
+
+    int storeFiles =  store.getStorefilesCount();
+    assertTrue(storeFiles > 0);
+
+    //verify the file exist before the chore
+    assertTrue(store.getFileSystem().exists(junkFilePath));
+
+    //set a 5 sec ttl
+    rs.getConfiguration().set(BrokenStoreFileCleaner.BROKEN_STOREFILE_CLEANER_TTL, "5000");
+    BrokenStoreFileCleaner cleaner = new BrokenStoreFileCleaner(15000000,
+      0, rs, rs.getConfiguration(), rs);
+    cleaner.chore();
+    //file is still present after chore run
+    assertTrue(store.getFileSystem().exists(junkFilePath));
+    Thread.sleep(5000);
+    cleaner.chore();
+    assertFalse(store.getFileSystem().exists(junkFilePath));
+
+    //verify no storefile got deleted
+    int currentStoreFiles =  store.getStorefilesCount();
+    assertEquals(currentStoreFiles, storeFiles);
+  }
+
+  private Table createTableWithData(TableName tableName) throws IOException {
+    Table table = testUtil.createTable(tableName, fam);
+    try {
+      for (int i = 1; i < 10; i++) {
+        Put p = new Put(Bytes.toBytes("row" + i));
+        p.addColumn(fam, qual1, val);
+        table.put(p);
+      }
+      // flush them
+      testUtil.getAdmin().flush(tableName);
+      for (int i = 11; i < 20; i++) {
+        Put p = new Put(Bytes.toBytes("row" + i));
+        p.addColumn(fam, qual1, val);
+        table.put(p);
+      }
+      // flush them
+      testUtil.getAdmin().flush(tableName);
+      for (int i = 21; i < 30; i++) {
+        Put p = new Put(Bytes.toBytes("row" + i));
+        p.addColumn(fam, qual1, val);
+        table.put(p);
+      }
+      // flush them
+      testUtil.getAdmin().flush(tableName);
+    } catch (IOException e) {
+      table.close();
+      throw e;
+    }
+    return table;
+  }
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactorMemLeak.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactorMemLeak.java
index 9a6e96a..1b76c52 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactorMemLeak.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactorMemLeak.java
@@ -128,13 +128,13 @@ public class TestCompactorMemLeak {
     }
 
     @Override
-    protected List<Path> commitWriter(StoreFileWriter writer, FileDetails fd,
+    protected List<Path> commitWriter(FileDetails fd,
         CompactionRequestImpl request) throws IOException {
       HFileWriterImpl writerImpl = (HFileWriterImpl) writer.writer;
       Cell cell = writerImpl.getLastCell();
       // The cell should be backend with an KeyOnlyKeyValue.
       IS_LAST_CELL_ON_HEAP.set(cell instanceof KeyOnlyKeyValue);
-      return super.commitWriter(writer, fd, request);
+      return super.commitWriter(fd, request);
     }
   }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTracker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTracker.java
index b30ca47..9818972 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTracker.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTracker.java
@@ -47,7 +47,6 @@ public class TestStoreFileTracker extends DefaultStoreFileTracker {
     } else {
       LOG.info("ctx.getRegionFileSystem() returned null. Leaving storeId null.");
     }
-
   }
 
   @Override

[hbase] 14/15: HBASE-26454 CreateTableProcedure still relies on temp dir and renames… (#3845)

Posted by zh...@apache.org.
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

commit 48e28917e0c61a1166f0e356513d1c964f95664f
Author: Wellington Ramos Chevreuil <wc...@apache.org>
AuthorDate: Fri Nov 19 12:16:29 2021 +0000

    HBASE-26454 CreateTableProcedure still relies on temp dir and renames… (#3845)
    
    Signed-off-by: Duo Zhang <zh...@apache.org>
---
 .../master/procedure/CreateTableProcedure.java     |  30 +-----
 .../master/procedure/DeleteTableProcedure.java     | 115 +++++++--------------
 .../access/SnapshotScannerHDFSAclHelper.java       |   4 +-
 .../hadoop/hbase/master/TestMasterFileSystem.java  |  29 ++----
 .../master/procedure/TestDeleteTableProcedure.java |  66 ------------
 5 files changed, 53 insertions(+), 191 deletions(-)

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 55e3212..441fddb 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
@@ -22,7 +22,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;
 import org.apache.hadoop.hbase.HBaseIOException;
@@ -316,41 +315,22 @@ public class CreateTableProcedure
       final TableDescriptor tableDescriptor, List<RegionInfo> newRegions,
       final CreateHdfsRegions hdfsRegionHandler) throws IOException {
     final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
-    final Path tempdir = mfs.getTempDir();
 
     // 1. Create Table Descriptor
     // using a copy of descriptor, table will be created enabling first
-    final Path tempTableDir = CommonFSUtils.getTableDir(tempdir, tableDescriptor.getTableName());
+    final Path tableDir = CommonFSUtils.getTableDir(mfs.getRootDir(),
+      tableDescriptor.getTableName());
     ((FSTableDescriptors)(env.getMasterServices().getTableDescriptors()))
-        .createTableDescriptorForTableDirectory(tempTableDir, tableDescriptor, false);
+        .createTableDescriptorForTableDirectory(
+          tableDir, tableDescriptor, false);
 
     // 2. Create Regions
-    newRegions = hdfsRegionHandler.createHdfsRegions(env, tempdir,
+    newRegions = hdfsRegionHandler.createHdfsRegions(env, mfs.getRootDir(),
             tableDescriptor.getTableName(), newRegions);
 
-    // 3. Move Table temp directory to the hbase root location
-    moveTempDirectoryToHBaseRoot(env, tableDescriptor, tempTableDir);
-
     return newRegions;
   }
 
-  protected static void moveTempDirectoryToHBaseRoot(
-    final MasterProcedureEnv env,
-    final TableDescriptor tableDescriptor,
-    final Path tempTableDir) throws IOException {
-    final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
-    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);
-    }
-    if (!fs.rename(tempTableDir, tableDir)) {
-      throw new IOException("Unable to move table from temp=" + tempTableDir +
-        " to hbase root=" + tableDir);
-    }
-  }
-
   protected static List<RegionInfo> addTableToMeta(final MasterProcedureEnv env,
     final TableDescriptor tableDescriptor, final List<RegionInfo> regions) throws IOException {
     assert (regions != null && regions.size() > 0) : "expected at least 1 region, got " + regions;
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 8322383..4614486 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
@@ -20,10 +20,7 @@ package org.apache.hadoop.hbase.master.procedure;
 
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.List;
-import java.util.stream.Collectors;
-import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.MetaTableAccessor;
@@ -52,11 +49,12 @@ import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
+
 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;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteTableState;
-import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
 
 @InterfaceAudience.Private
 public class DeleteTableProcedure
@@ -278,92 +276,59 @@ public class DeleteTableProcedure
       final boolean archive) throws IOException {
     final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
     final FileSystem fs = mfs.getFileSystem();
-    final Path tempdir = mfs.getTempDir();
 
     final Path tableDir = CommonFSUtils.getTableDir(mfs.getRootDir(), tableName);
-    final Path tempTableDir = CommonFSUtils.getTableDir(tempdir, tableName);
 
     if (fs.exists(tableDir)) {
-      // Ensure temp exists
-      if (!fs.exists(tempdir) && !fs.mkdirs(tempdir)) {
-        throw new IOException("HBase temp directory '" + tempdir + "' creation failure.");
-      }
-
-      // Ensure parent exists
-      if (!fs.exists(tempTableDir.getParent()) && !fs.mkdirs(tempTableDir.getParent())) {
-        throw new IOException("HBase temp directory '" + tempdir + "' creation failure.");
+      // Archive regions from FS (temp directory)
+      if (archive) {
+        List<Path> regionDirList = new ArrayList<>();
+        for (RegionInfo region : regions) {
+          if (RegionReplicaUtil.isDefaultReplica(region)) {
+            regionDirList.add(FSUtils.getRegionDirFromTableDir(tableDir, region));
+            List<RegionInfo> mergeRegions = MetaTableAccessor
+              .getMergeRegions(env.getMasterServices().getConnection(), region.getRegionName());
+            if (!CollectionUtils.isEmpty(mergeRegions)) {
+              mergeRegions.stream().forEach(
+                r -> regionDirList.add(FSUtils.getRegionDirFromTableDir(tableDir, r)));
+            }
+          }
+        }
+        HFileArchiver
+          .archiveRegions(env.getMasterConfiguration(), fs, mfs.getRootDir(), tableDir,
+            regionDirList);
+        if (!regionDirList.isEmpty()) {
+          LOG.debug("Archived {} regions", tableName);
+        }
       }
 
-      if (fs.exists(tempTableDir)) {
-        // TODO
-        // what's in this dir? something old? probably something manual from the user...
-        // let's get rid of this stuff...
-        FileStatus[] files = fs.listStatus(tempTableDir);
-        if (files != null && files.length > 0) {
-          List<Path> regionDirList = Arrays.stream(files)
-            .filter(FileStatus::isDirectory)
-            .map(FileStatus::getPath)
-            .collect(Collectors.toList());
-          HFileArchiver.archiveRegions(env.getMasterConfiguration(), fs, mfs.getRootDir(),
-            tempTableDir, regionDirList);
-        }
-        fs.delete(tempTableDir, true);
+      // Archive mob data
+      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)) {
+        HFileArchiver.archiveRegion(fs, mfs.getRootDir(), mobTableDir, regionDir);
       }
 
-      // Move the table in /hbase/.tmp
-      if (!fs.rename(tableDir, tempTableDir)) {
-        throw new IOException("Unable to move '" + tableDir + "' to temp '" + tempTableDir + "'");
+      // Delete table directory from FS
+      if (!fs.delete(tableDir, true) && fs.exists(tableDir)) {
+        throw new IOException("Couldn't delete " + tableDir);
       }
-    }
 
-    // Archive regions from FS (temp directory)
-    if (archive) {
-      List<Path> regionDirList = new ArrayList<>();
-      for (RegionInfo region : regions) {
-        if (RegionReplicaUtil.isDefaultReplica(region)) {
-          regionDirList.add(FSUtils.getRegionDirFromTableDir(tempTableDir, region));
-          List<RegionInfo> mergeRegions = MetaTableAccessor
-              .getMergeRegions(env.getMasterServices().getConnection(), region.getRegionName());
-          if (!CollectionUtils.isEmpty(mergeRegions)) {
-            mergeRegions.stream()
-                .forEach(r -> regionDirList.add(FSUtils.getRegionDirFromTableDir(tempTableDir, r)));
-          }
+      // Delete the table directory where the mob files are saved
+      if (mobTableDir != null && fs.exists(mobTableDir)) {
+        if (!fs.delete(mobTableDir, true)) {
+          throw new IOException("Couldn't delete mob dir " + mobTableDir);
         }
       }
-      HFileArchiver.archiveRegions(env.getMasterConfiguration(), fs, mfs.getRootDir(), tempTableDir,
-        regionDirList);
-      if (!regionDirList.isEmpty()) {
-        LOG.debug("Archived {} regions", tableName);
-      }
-    }
-
-    // Archive mob data
-    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)) {
-      HFileArchiver.archiveRegion(fs, mfs.getRootDir(), mobTableDir, regionDir);
-    }
-
-    // Delete table directory from FS (temp directory)
-    if (!fs.delete(tempTableDir, true) && fs.exists(tempTableDir)) {
-      throw new IOException("Couldn't delete " + tempTableDir);
-    }
 
-    // Delete the table directory where the mob files are saved
-    if (mobTableDir != null && fs.exists(mobTableDir)) {
-      if (!fs.delete(mobTableDir, true)) {
-        throw new IOException("Couldn't delete mob dir " + mobTableDir);
+      // Delete the directory on wal filesystem
+      FileSystem walFs = mfs.getWALFileSystem();
+      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);
       }
     }
-
-    // Delete the directory on wal filesystem
-    FileSystem walFs = mfs.getWALFileSystem();
-    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/security/access/SnapshotScannerHDFSAclHelper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/SnapshotScannerHDFSAclHelper.java
index ffe8dab..fbdc638 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/SnapshotScannerHDFSAclHelper.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/SnapshotScannerHDFSAclHelper.java
@@ -478,8 +478,8 @@ public class SnapshotScannerHDFSAclHelper implements Closeable {
    */
   List<Path> getTableRootPaths(TableName tableName, boolean includeSnapshotPath)
       throws IOException {
-    List<Path> paths = Lists.newArrayList(pathHelper.getTmpTableDir(tableName),
-      pathHelper.getDataTableDir(tableName), pathHelper.getMobTableDir(tableName),
+    List<Path> paths = Lists.newArrayList(pathHelper.getDataTableDir(tableName),
+      pathHelper.getMobTableDir(tableName),
       pathHelper.getArchiveTableDir(tableName));
     if (includeSnapshotPath) {
       paths.addAll(getTableSnapshotPaths(tableName));
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 63d303d..1461c06 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
@@ -18,8 +18,7 @@
 package org.apache.hadoop.hbase.master;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.junit.Assert.assertFalse;
 
 import java.util.List;
 import org.apache.hadoop.fs.FileSystem;
@@ -33,7 +32,6 @@ 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.HFileArchiveTestingUtil;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
@@ -85,7 +83,7 @@ public class TestMasterFileSystem {
   }
 
   @Test
-  public void testCheckTempDir() throws Exception {
+  public void testCheckNoTempDir() throws Exception {
     final MasterFileSystem masterFileSystem =
       UTIL.getMiniHBaseCluster().getMaster().getMasterFileSystem();
 
@@ -110,28 +108,13 @@ public class TestMasterFileSystem {
     // disable the table so that we can manipulate the files
     UTIL.getAdmin().disableTable(tableName);
 
-    final Path tableDir = CommonFSUtils.getTableDir(masterFileSystem.getRootDir(), tableName);
     final Path tempDir = masterFileSystem.getTempDir();
-    final Path tempTableDir = CommonFSUtils.getTableDir(tempDir, tableName);
+    final Path tempNsDir = CommonFSUtils.getNamespaceDir(tempDir,
+      tableName.getNamespaceAsString());
     final FileSystem fs = masterFileSystem.getFileSystem();
 
-    // move the table to the temporary directory
-    if (!fs.rename(tableDir, tempTableDir)) {
-      fail();
-    }
-
-    masterFileSystem.checkTempDir(tempDir, UTIL.getConfiguration(), fs);
-
-    // check if the temporary directory exists and is empty
-    assertTrue(fs.exists(tempDir));
-    assertEquals(0, fs.listStatus(tempDir).length);
-
-    // check for the existence of the archive directory
-    for (HRegion region : regions) {
-      Path archiveDir = HFileArchiveTestingUtil.getRegionArchiveDir(UTIL.getConfiguration(),
-        region);
-      assertTrue(fs.exists(archiveDir));
-    }
+    // checks the temporary directory does not exist
+    assertFalse(fs.exists(tempNsDir));
 
     UTIL.deleteTable(tableName);
   }
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 1dd7dc4..9367a57 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
@@ -17,34 +17,23 @@
  */
 package org.apache.hadoop.hbase.master.procedure;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
-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;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotDisabledException;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.client.RegionInfo;
-import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.master.MasterFileSystem;
 import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
-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.HFileArchiveTestingUtil;
 import org.junit.ClassRule;
 import org.junit.Rule;
 import org.junit.Test;
@@ -186,59 +175,4 @@ public class TestDeleteTableProcedure extends TestTableDDLProcedureBase {
 
     MasterProcedureTestingUtility.validateTableDeletion(getMaster(), tableName);
   }
-
-  @Test
-  public void testDeleteWhenTempDirIsNotEmpty() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
-    final String FAM = "fam";
-    final byte[][] splitKeys = new byte[][] {
-      Bytes.toBytes("b"), Bytes.toBytes("c"), Bytes.toBytes("d")
-    };
-
-    // create the table
-    MasterProcedureTestingUtility.createTable(
-      getMasterProcedureExecutor(), tableName, splitKeys, FAM);
-
-    // get the current store files for the regions
-    List<HRegion> regions = UTIL.getHBaseCluster().getRegions(tableName);
-    // make sure we have 4 regions serving this table
-    assertEquals(4, regions.size());
-
-    // load the table
-    try (Table table = UTIL.getConnection().getTable(tableName)) {
-      UTIL.loadTable(table, Bytes.toBytes(FAM));
-    }
-
-    // disable the table so that we can manipulate the files
-    UTIL.getAdmin().disableTable(tableName);
-
-    final MasterFileSystem masterFileSystem =
-      UTIL.getMiniHBaseCluster().getMaster().getMasterFileSystem();
-    final Path tableDir = CommonFSUtils.getTableDir(masterFileSystem.getRootDir(), tableName);
-    final Path tempDir = masterFileSystem.getTempDir();
-    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
-    if (!FileUtil.copy(fs, tableDir, fs, tempTableDir, false, UTIL.getConfiguration())) {
-      fail();
-    }
-
-    // delete the table
-    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
-    long procId = ProcedureTestingUtility.submitAndWait(procExec,
-      new DeleteTableProcedure(procExec.getEnvironment(), tableName));
-    ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
-    MasterProcedureTestingUtility.validateTableDeletion(getMaster(), tableName);
-
-    // check if the temporary directory is deleted
-    assertFalse(fs.exists(tempTableDir));
-
-    // check for the existence of the archive directory
-    for (HRegion region : regions) {
-      Path archiveDir = HFileArchiveTestingUtil.getRegionArchiveDir(UTIL.getConfiguration(),
-        region);
-      assertTrue(fs.exists(archiveDir));
-    }
-  }
 }

[hbase] 15/15: HBASE-26286: Add support for specifying store file tracker when restoring or cloning snapshot

Posted by zh...@apache.org.
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

commit 53285d8e2a81049a35e367142a3616190df16b2d
Author: BukrosSzabolcs <bu...@gmail.com>
AuthorDate: Wed Dec 15 20:09:03 2021 -0500

    HBASE-26286: Add support for specifying store file tracker when restoring or cloning snapshot
    
    Closes #3851
    
    Signed-off-by: Duo Zhang <zh...@apache.org>
    Signed-off-by: Josh Elser <el...@apache.org>
---
 .../java/org/apache/hadoop/hbase/client/Admin.java | 42 ++++++++++++-
 .../org/apache/hadoop/hbase/client/AsyncAdmin.java | 14 ++++-
 .../hadoop/hbase/client/AsyncHBaseAdmin.java       |  6 +-
 .../hbase/client/ColumnFamilyDescriptor.java       |  5 ++
 .../client/ColumnFamilyDescriptorBuilder.java      |  6 ++
 .../org/apache/hadoop/hbase/client/HBaseAdmin.java | 26 ++++----
 .../hadoop/hbase/client/RawAsyncHBaseAdmin.java    | 24 +++++---
 .../src/main/protobuf/Master.proto                 |  1 +
 .../src/main/protobuf/MasterProcedure.proto        |  1 +
 .../org/apache/hadoop/hbase/master/HMaster.java    | 25 ++++----
 .../hadoop/hbase/master/MasterRpcServices.java     |  2 +-
 .../master/procedure/CloneSnapshotProcedure.java   | 54 +++++++++++++++-
 .../master/procedure/RestoreSnapshotProcedure.java |  7 ++-
 .../hbase/master/snapshot/SnapshotManager.java     | 27 +++++---
 .../storefiletracker/StoreFileTrackerFactory.java  | 39 +++++++++++-
 .../hbase/snapshot/RestoreSnapshotHelper.java      | 11 ++--
 .../TestCloneSnapshotFromClientCustomSFT.java      | 72 ++++++++++++++++++++++
 .../storefiletracker/TestStoreFileTracker.java     |  2 -
 .../TestStoreFileTrackerFactory.java               | 52 ++++++++++++++++
 hbase-shell/src/main/ruby/hbase/admin.rb           |  4 +-
 hbase-shell/src/main/ruby/hbase_constants.rb       |  1 +
 .../src/main/ruby/shell/commands/clone_snapshot.rb |  6 +-
 .../hadoop/hbase/thrift2/client/ThriftAdmin.java   |  4 +-
 23 files changed, 365 insertions(+), 66 deletions(-)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
index a3a5107..36ebc9e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
@@ -70,6 +70,7 @@ import org.apache.hadoop.hbase.util.Pair;
 import org.apache.yetus.audience.InterfaceAudience;
 
 import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList;
+import org.apache.yetus.audience.InterfaceStability;
 
 /**
  * The administrative API for HBase. Obtain an instance from {@link Connection#getAdmin()} and
@@ -2410,7 +2411,25 @@ public interface Admin extends Abortable, Closeable {
    */
   default void cloneSnapshot(String snapshotName, TableName tableName)
       throws IOException, TableExistsException, RestoreSnapshotException {
-    cloneSnapshot(snapshotName, tableName, false);
+    cloneSnapshot(snapshotName, tableName, false, null);
+  }
+
+  /**
+   * Create a new table by cloning the snapshot content.
+   * @param snapshotName name of the snapshot to be cloned
+   * @param tableName name of the table where the snapshot will be restored
+   * @param restoreAcl <code>true</code> to clone acl into newly created table
+   * @param customSFT specify the StoreFileTracker used for the table
+   * @throws IOException if a remote or network exception occurs
+   * @throws TableExistsException if table to be created already exists
+   * @throws RestoreSnapshotException if snapshot failed to be cloned
+   * @throws IllegalArgumentException if the specified table has not a valid name
+   */
+  default void cloneSnapshot(String snapshotName, TableName tableName, boolean restoreAcl,
+    String customSFT)
+    throws IOException, TableExistsException, RestoreSnapshotException {
+    get(cloneSnapshotAsync(snapshotName, tableName, restoreAcl, customSFT), getSyncWaitTimeout(),
+      TimeUnit.MILLISECONDS);
   }
 
   /**
@@ -2457,8 +2476,25 @@ public interface Admin extends Abortable, Closeable {
    * @throws RestoreSnapshotException if snapshot failed to be cloned
    * @throws IllegalArgumentException if the specified table has not a valid name
    */
-  Future<Void> cloneSnapshotAsync(String snapshotName, TableName tableName, boolean restoreAcl)
-      throws IOException, TableExistsException, RestoreSnapshotException;
+  default Future<Void> cloneSnapshotAsync(String snapshotName, TableName tableName,
+    boolean restoreAcl)
+      throws IOException, TableExistsException, RestoreSnapshotException {
+    return cloneSnapshotAsync(snapshotName, tableName, restoreAcl, null);
+  }
+
+  /**
+   * Create a new table by cloning the snapshot content.
+   * @param snapshotName name of the snapshot to be cloned
+   * @param tableName name of the table where the snapshot will be restored
+   * @param restoreAcl <code>true</code> to clone acl into newly created table
+   * @param customSFT specify the StroreFileTracker used for the table
+   * @throws IOException if a remote or network exception occurs
+   * @throws TableExistsException if table to be created already exists
+   * @throws RestoreSnapshotException if snapshot failed to be cloned
+   * @throws IllegalArgumentException if the specified table has not a valid name
+   */
+  Future<Void> cloneSnapshotAsync(String snapshotName, TableName tableName, boolean restoreAcl,
+    String customSFT) throws IOException, TableExistsException, RestoreSnapshotException;
 
   /**
    * Execute a distributed procedure on a cluster.
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
index 85d5455..42c3f04 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
@@ -872,8 +872,20 @@ public interface AsyncAdmin {
    * @param tableName name of the table where the snapshot will be restored
    * @param restoreAcl <code>true</code> to restore acl of snapshot
    */
+  default CompletableFuture<Void> cloneSnapshot(String snapshotName, TableName tableName,
+      boolean restoreAcl) {
+    return cloneSnapshot(snapshotName, tableName, restoreAcl, null);
+  }
+
+  /**
+   * Create a new table by cloning the snapshot content.
+   * @param snapshotName name of the snapshot to be cloned
+   * @param tableName name of the table where the snapshot will be restored
+   * @param restoreAcl <code>true</code> to restore acl of snapshot
+   * @param customSFT specify the StroreFileTracker used for the table
+   */
   CompletableFuture<Void> cloneSnapshot(String snapshotName, TableName tableName,
-      boolean restoreAcl);
+      boolean restoreAcl, String customSFT);
 
   /**
    * List completed snapshots.
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
index db720f3..16ebd15 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
@@ -483,14 +483,14 @@ class AsyncHBaseAdmin implements AsyncAdmin {
 
   @Override
   public CompletableFuture<Void> restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot,
-      boolean restoreAcl) {
+    boolean restoreAcl) {
     return wrap(rawAdmin.restoreSnapshot(snapshotName, takeFailSafeSnapshot, restoreAcl));
   }
 
   @Override
   public CompletableFuture<Void> cloneSnapshot(String snapshotName, TableName tableName,
-      boolean restoreAcl) {
-    return wrap(rawAdmin.cloneSnapshot(snapshotName, tableName, restoreAcl));
+      boolean restoreAcl, String customSFT) {
+    return wrap(rawAdmin.cloneSnapshot(snapshotName, tableName, restoreAcl, customSFT));
   }
 
   @Override
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ColumnFamilyDescriptor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ColumnFamilyDescriptor.java
index 86d561d..001d672 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ColumnFamilyDescriptor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ColumnFamilyDescriptor.java
@@ -199,6 +199,11 @@ public interface ColumnFamilyDescriptor {
    * @param key The key.
    * @return A clone value. Null if no mapping for the key
    */
+  String getValue(String key);
+  /**
+   * @param key The key.
+   * @return A clone value. Null if no mapping for the key
+   */
   byte[] getValue(byte[] key);
   /**
    * It clone all bytes of all elements.
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ColumnFamilyDescriptorBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ColumnFamilyDescriptorBuilder.java
index d1c9264..5dccd0b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ColumnFamilyDescriptorBuilder.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ColumnFamilyDescriptorBuilder.java
@@ -674,6 +674,12 @@ public class ColumnFamilyDescriptorBuilder {
     }
 
     @Override
+    public String getValue(String key) {
+      Bytes rval = values.get(new Bytes(Bytes.toBytes(key)));
+      return rval == null ? null : Bytes.toString(rval.get(), rval.getOffset(), rval.getLength());
+    }
+
+    @Override
     public Map<Bytes, Bytes> getValues() {
       return Collections.unmodifiableMap(values);
     }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
index 0704d51..b860bdc 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
@@ -108,9 +108,11 @@ 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.base.Preconditions;
 import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
 import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos;
@@ -2628,7 +2630,7 @@ public class HBaseAdmin implements Admin {
     try {
       // Restore snapshot
       get(
-        internalRestoreSnapshotAsync(snapshotName, tableName, restoreAcl),
+        internalRestoreSnapshotAsync(snapshotName, tableName, restoreAcl, null),
         syncWaitTimeout,
         TimeUnit.MILLISECONDS);
     } catch (IOException e) {
@@ -2637,7 +2639,7 @@ public class HBaseAdmin implements Admin {
       if (takeFailSafeSnapshot) {
         try {
           get(
-            internalRestoreSnapshotAsync(failSafeSnapshotSnapshotName, tableName, restoreAcl),
+            internalRestoreSnapshotAsync(failSafeSnapshotSnapshotName, tableName, restoreAcl, null),
             syncWaitTimeout,
             TimeUnit.MILLISECONDS);
           String msg = "Restore snapshot=" + snapshotName +
@@ -2680,16 +2682,17 @@ public class HBaseAdmin implements Admin {
       throw new TableNotDisabledException(tableName);
     }
 
-    return internalRestoreSnapshotAsync(snapshotName, tableName, false);
+    return internalRestoreSnapshotAsync(snapshotName, tableName, false, null);
   }
 
   @Override
   public Future<Void> cloneSnapshotAsync(String snapshotName, TableName tableName,
-      boolean restoreAcl) throws IOException, TableExistsException, RestoreSnapshotException {
+    boolean restoreAcl, String customSFT)
+    throws IOException, TableExistsException, RestoreSnapshotException {
     if (tableExists(tableName)) {
       throw new TableExistsException(tableName);
     }
-    return internalRestoreSnapshotAsync(snapshotName, tableName, restoreAcl);
+    return internalRestoreSnapshotAsync(snapshotName, tableName, restoreAcl, customSFT);
   }
 
   @Override
@@ -2778,7 +2781,7 @@ public class HBaseAdmin implements Admin {
    * @throws IllegalArgumentException if the restore request is formatted incorrectly
    */
   private Future<Void> internalRestoreSnapshotAsync(final String snapshotName,
-      final TableName tableName, final boolean restoreAcl)
+      final TableName tableName, final boolean restoreAcl, String customSFT)
       throws IOException, RestoreSnapshotException {
     final SnapshotProtos.SnapshotDescription snapshot =
         SnapshotProtos.SnapshotDescription.newBuilder()
@@ -2793,13 +2796,15 @@ public class HBaseAdmin implements Admin {
           Long nonce = ng.newNonce();
       @Override
       protected RestoreSnapshotResponse rpcCall() throws Exception {
-        final RestoreSnapshotRequest request = RestoreSnapshotRequest.newBuilder()
+        final RestoreSnapshotRequest.Builder builder = RestoreSnapshotRequest.newBuilder()
             .setSnapshot(snapshot)
             .setNonceGroup(nonceGroup)
             .setNonce(nonce)
-            .setRestoreACL(restoreAcl)
-            .build();
-        return master.restoreSnapshot(getRpcController(), request);
+            .setRestoreACL(restoreAcl);
+        if (customSFT != null) {
+          builder.setCustomSFT(customSFT);
+        }
+        return master.restoreSnapshot(getRpcController(), builder.build());
       }
     });
 
@@ -4468,5 +4473,4 @@ public class HBaseAdmin implements Admin {
       }
     });
   }
-
 }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
index a0e5320..64b82cd 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
@@ -1936,7 +1936,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
         } else if (!exists) {
           // if table does not exist, then just clone snapshot into new table.
           completeConditionalOnFuture(future,
-            internalRestoreSnapshot(snapshotName, finalTableName, restoreAcl));
+            internalRestoreSnapshot(snapshotName, finalTableName, restoreAcl, null));
         } else {
           addListener(isTableDisabled(finalTableName), (disabled, err4) -> {
             if (err4 != null) {
@@ -1972,12 +1972,13 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
           future.completeExceptionally(err);
         } else {
           // Step.2 Restore snapshot
-          addListener(internalRestoreSnapshot(snapshotName, tableName, restoreAcl),
+          addListener(internalRestoreSnapshot(snapshotName, tableName, restoreAcl, null),
             (void2, err2) -> {
               if (err2 != null) {
                 // Step.3.a Something went wrong during the restore and try to rollback.
                 addListener(
-                  internalRestoreSnapshot(failSafeSnapshotSnapshotName, tableName, restoreAcl),
+                  internalRestoreSnapshot(failSafeSnapshotSnapshotName, tableName, restoreAcl,
+                    null),
                   (void3, err3) -> {
                     if (err3 != null) {
                       future.completeExceptionally(err3);
@@ -2007,7 +2008,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
       });
       return future;
     } else {
-      return internalRestoreSnapshot(snapshotName, tableName, restoreAcl);
+      return internalRestoreSnapshot(snapshotName, tableName, restoreAcl, null);
     }
   }
 
@@ -2024,7 +2025,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
 
   @Override
   public CompletableFuture<Void> cloneSnapshot(String snapshotName, TableName tableName,
-      boolean restoreAcl) {
+      boolean restoreAcl, String customSFT) {
     CompletableFuture<Void> future = new CompletableFuture<>();
     addListener(tableExists(tableName), (exists, err) -> {
       if (err != null) {
@@ -2033,14 +2034,14 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
         future.completeExceptionally(new TableExistsException(tableName));
       } else {
         completeConditionalOnFuture(future,
-          internalRestoreSnapshot(snapshotName, tableName, restoreAcl));
+          internalRestoreSnapshot(snapshotName, tableName, restoreAcl, customSFT));
       }
     });
     return future;
   }
 
   private CompletableFuture<Void> internalRestoreSnapshot(String snapshotName, TableName tableName,
-      boolean restoreAcl) {
+      boolean restoreAcl, String customSFT) {
     SnapshotProtos.SnapshotDescription snapshot = SnapshotProtos.SnapshotDescription.newBuilder()
       .setName(snapshotName).setTable(tableName.getNameAsString()).build();
     try {
@@ -2048,10 +2049,15 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
     } catch (IllegalArgumentException e) {
       return failedFuture(e);
     }
+    RestoreSnapshotRequest.Builder builder =
+      RestoreSnapshotRequest.newBuilder().setSnapshot(snapshot).setNonceGroup(ng.getNonceGroup())
+        .setNonce(ng.newNonce()).setRestoreACL(restoreAcl);
+    if(customSFT != null){
+      builder.setCustomSFT(customSFT);
+    }
     return waitProcedureResult(this.<Long> newMasterCaller().action((controller, stub) -> this
       .<RestoreSnapshotRequest, RestoreSnapshotResponse, Long> call(controller, stub,
-        RestoreSnapshotRequest.newBuilder().setSnapshot(snapshot).setNonceGroup(ng.getNonceGroup())
-          .setNonce(ng.newNonce()).setRestoreACL(restoreAcl).build(),
+        builder.build(),
         (s, c, req, done) -> s.restoreSnapshot(c, req, done), (resp) -> resp.getProcId()))
       .call());
   }
diff --git a/hbase-protocol-shaded/src/main/protobuf/Master.proto b/hbase-protocol-shaded/src/main/protobuf/Master.proto
index 4a6bb39..8eccb7a 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Master.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Master.proto
@@ -461,6 +461,7 @@ message RestoreSnapshotRequest {
   optional uint64 nonce_group = 2 [default = 0];
   optional uint64 nonce = 3 [default = 0];
   optional bool restoreACL = 4 [default = false];
+  optional string customSFT = 5;
 }
 
 message RestoreSnapshotResponse {
diff --git a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
index 66d1640..cdc6e73 100644
--- a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
@@ -208,6 +208,7 @@ message CloneSnapshotStateData {
   repeated RegionInfo region_info = 4;
   repeated RestoreParentToChildRegionsPair parent_to_child_regions_pair_list = 5;
   optional bool restore_acl = 6;
+  optional string customSFT = 7;
 }
 
 enum RestoreSnapshotState {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index e67ab53..8455581 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -2593,8 +2593,8 @@ public class HMaster extends HRegionServer implements MasterServices {
 
   }
 
-  public long restoreSnapshot(final SnapshotDescription snapshotDesc,
-      final long nonceGroup, final long nonce, final boolean restoreAcl) throws IOException {
+  public long restoreSnapshot(final SnapshotDescription snapshotDesc, final long nonceGroup,
+    final long nonce, final boolean restoreAcl, final String customSFT) throws IOException {
     checkInitialized();
     getSnapshotManager().checkSnapshotSupport();
 
@@ -2603,18 +2603,19 @@ public class HMaster extends HRegionServer implements MasterServices {
     getClusterSchema().getNamespace(dstTable.getNamespaceAsString());
 
     return MasterProcedureUtil.submitProcedure(
-        new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
-      @Override
-      protected void run() throws IOException {
+      new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
+        @Override
+        protected void run() throws IOException {
           setProcId(
-            getSnapshotManager().restoreOrCloneSnapshot(snapshotDesc, getNonceKey(), restoreAcl));
-      }
+            getSnapshotManager().restoreOrCloneSnapshot(snapshotDesc, getNonceKey(), restoreAcl,
+              customSFT));
+        }
 
-      @Override
-      protected String getDescription() {
-        return "RestoreSnapshotProcedure";
-      }
-    });
+        @Override
+        protected String getDescription() {
+          return "RestoreSnapshotProcedure";
+        }
+      });
   }
 
   private void checkTableExists(final TableName tableName)
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index a64c990..695c960 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -1514,7 +1514,7 @@ public class MasterRpcServices extends RSRpcServices implements
       RestoreSnapshotRequest request) throws ServiceException {
     try {
       long procId = master.restoreSnapshot(request.getSnapshot(), request.getNonceGroup(),
-        request.getNonce(), request.getRestoreACL());
+        request.getNonce(), request.getRestoreACL(), request.getCustomSFT());
       return RestoreSnapshotResponse.newBuilder().setProcId(procId).build();
     } catch (ForeignException e) {
       throw new ServiceException(e.getCause());
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 7157fbf..f6185d1 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
@@ -30,6 +30,8 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.TableExistsException;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
@@ -44,6 +46,8 @@ import org.apache.hadoop.hbase.master.procedure.CreateTableProcedure.CreateHdfsR
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
 import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
+import org.apache.hadoop.hbase.procedure2.util.StringUtils;
+import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory;
 import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
 import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
 import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper;
@@ -72,6 +76,7 @@ public class CloneSnapshotProcedure
   private TableDescriptor tableDescriptor;
   private SnapshotDescription snapshot;
   private boolean restoreAcl;
+  private String customSFT;
   private List<RegionInfo> newRegions = null;
   private Map<String, Pair<String, String> > parentsToChildrenPairMap = new HashMap<>();
 
@@ -96,12 +101,19 @@ public class CloneSnapshotProcedure
    * @param snapshot snapshot to clone from
    */
   public CloneSnapshotProcedure(final MasterProcedureEnv env,
+    final TableDescriptor tableDescriptor, final SnapshotDescription snapshot,
+    final boolean restoreAcl) {
+    this(env, tableDescriptor, snapshot, restoreAcl, null);
+  }
+
+  public CloneSnapshotProcedure(final MasterProcedureEnv env,
       final TableDescriptor tableDescriptor, final SnapshotDescription snapshot,
-      final boolean restoreAcl) {
+      final boolean restoreAcl, final String customSFT) {
     super(env);
     this.tableDescriptor = tableDescriptor;
     this.snapshot = snapshot;
     this.restoreAcl = restoreAcl;
+    this.customSFT = customSFT;
 
     getMonitorStatus();
   }
@@ -139,6 +151,7 @@ public class CloneSnapshotProcedure
           setNextState(CloneSnapshotState.CLONE_SNAPSHOT_WRITE_FS_LAYOUT);
           break;
         case CLONE_SNAPSHOT_WRITE_FS_LAYOUT:
+          updateTableDescriptorWithSFT();
           newRegions = createFilesystemLayout(env, tableDescriptor, newRegions);
           env.getMasterServices().getTableDescriptors().update(tableDescriptor, true);
           setNextState(CloneSnapshotState.CLONE_SNAPSHOT_ADD_TO_META);
@@ -203,6 +216,37 @@ public class CloneSnapshotProcedure
     return Flow.HAS_MORE_STATE;
   }
 
+  /**
+   * If a StoreFileTracker is specified we strip the TableDescriptor from previous SFT config
+   * and set the specified SFT on the table level
+   */
+  private void updateTableDescriptorWithSFT() {
+    if (StringUtils.isEmpty(customSFT)){
+      return;
+    }
+
+    TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableDescriptor);
+    builder.setValue(StoreFileTrackerFactory.TRACKER_IMPL, customSFT);
+    for (ColumnFamilyDescriptor family : tableDescriptor.getColumnFamilies()){
+      ColumnFamilyDescriptorBuilder cfBuilder = ColumnFamilyDescriptorBuilder.newBuilder(family);
+      cfBuilder.setConfiguration(StoreFileTrackerFactory.TRACKER_IMPL, null);
+      cfBuilder.setValue(StoreFileTrackerFactory.TRACKER_IMPL, null);
+      builder.modifyColumnFamily(cfBuilder.build());
+    }
+    tableDescriptor = builder.build();
+  }
+
+  private void validateSFT() {
+    if (StringUtils.isEmpty(customSFT)){
+      return;
+    }
+
+    //if customSFT is invalid getTrackerClass will throw a RuntimeException
+    Configuration sftConfig = new Configuration();
+    sftConfig.set(StoreFileTrackerFactory.TRACKER_IMPL, customSFT);
+    StoreFileTrackerFactory.getTrackerClass(sftConfig);
+  }
+
   @Override
   protected void rollbackState(final MasterProcedureEnv env, final CloneSnapshotState state)
       throws IOException {
@@ -292,6 +336,9 @@ public class CloneSnapshotProcedure
         cloneSnapshotMsg.addParentToChildRegionsPairList(parentToChildrenPair);
       }
     }
+    if (!StringUtils.isEmpty(customSFT)){
+      cloneSnapshotMsg.setCustomSFT(customSFT);
+    }
     serializer.serialize(cloneSnapshotMsg.build());
   }
 
@@ -327,6 +374,9 @@ public class CloneSnapshotProcedure
             parentToChildrenPair.getChild2RegionName()));
       }
     }
+    if (!StringUtils.isEmpty(cloneSnapshotMsg.getCustomSFT())){
+      customSFT = cloneSnapshotMsg.getCustomSFT();
+    }
     // Make sure that the monitor status is set up
     getMonitorStatus();
   }
@@ -340,6 +390,8 @@ public class CloneSnapshotProcedure
     if (env.getMasterServices().getTableDescriptors().exists(tableName)) {
       throw new TableExistsException(tableName);
     }
+
+    validateSFT();
   }
 
   /**
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RestoreSnapshotProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RestoreSnapshotProcedure.java
index e944062..6b28173 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RestoreSnapshotProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RestoreSnapshotProcedure.java
@@ -25,6 +25,7 @@ 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;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
@@ -89,6 +90,7 @@ public class RestoreSnapshotProcedure
   throws HBaseIOException {
     this(env, tableDescriptor, snapshot, false);
   }
+
   /**
    * Constructor
    * @param env MasterProcedureEnv
@@ -386,14 +388,15 @@ public class RestoreSnapshotProcedure
     FileSystem fs = fileSystemManager.getFileSystem();
     Path rootDir = fileSystemManager.getRootDir();
     final ForeignExceptionDispatcher monitorException = new ForeignExceptionDispatcher();
+    final Configuration conf = new Configuration(env.getMasterConfiguration());
 
     LOG.info("Starting restore snapshot=" + ClientSnapshotDescriptionUtils.toString(snapshot));
     try {
       Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshot, rootDir);
       SnapshotManifest manifest = SnapshotManifest.open(
-        env.getMasterServices().getConfiguration(), fs, snapshotDir, snapshot);
+        conf, fs, snapshotDir, snapshot);
       RestoreSnapshotHelper restoreHelper = new RestoreSnapshotHelper(
-        env.getMasterServices().getConfiguration(),
+        conf,
         fs,
         manifest,
         modifiedTableDescriptor,
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 3e8741f..b35a3df 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
@@ -66,6 +66,7 @@ import org.apache.hadoop.hbase.procedure.ProcedureCoordinator;
 import org.apache.hadoop.hbase.procedure.ProcedureCoordinatorRpcs;
 import org.apache.hadoop.hbase.procedure.ZKProcedureCoordinator;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
+import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory;
 import org.apache.hadoop.hbase.security.AccessDeniedException;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.access.AccessChecker;
@@ -751,8 +752,8 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
    * @throws IOException
    */
   private long cloneSnapshot(final SnapshotDescription reqSnapshot, final TableName tableName,
-      final SnapshotDescription snapshot, final TableDescriptor snapshotTableDesc,
-      final NonceKey nonceKey, final boolean restoreAcl) throws IOException {
+    final SnapshotDescription snapshot, final TableDescriptor snapshotTableDesc,
+    final NonceKey nonceKey, final boolean restoreAcl, final String customSFT) throws IOException {
     MasterCoprocessorHost cpHost = master.getMasterCoprocessorHost();
     TableDescriptor htd = TableDescriptorBuilder.copy(tableName, snapshotTableDesc);
     org.apache.hadoop.hbase.client.SnapshotDescription snapshotPOJO = null;
@@ -762,7 +763,7 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
     }
     long procId;
     try {
-      procId = cloneSnapshot(snapshot, htd, nonceKey, restoreAcl);
+      procId = cloneSnapshot(snapshot, htd, nonceKey, restoreAcl, customSFT);
     } catch (IOException e) {
       LOG.error("Exception occurred while cloning the snapshot " + snapshot.getName()
         + " as table " + tableName.getNameAsString(), e);
@@ -786,7 +787,8 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
    * @return procId the ID of the clone snapshot procedure
    */
   synchronized long cloneSnapshot(final SnapshotDescription snapshot,
-      final TableDescriptor tableDescriptor, final NonceKey nonceKey, final boolean restoreAcl)
+    final TableDescriptor tableDescriptor, final NonceKey nonceKey, final boolean restoreAcl,
+    final String customSFT)
       throws HBaseSnapshotException {
     TableName tableName = tableDescriptor.getTableName();
 
@@ -803,7 +805,7 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
     try {
       long procId = master.getMasterProcedureExecutor().submitProcedure(
         new CloneSnapshotProcedure(master.getMasterProcedureExecutor().getEnvironment(),
-                tableDescriptor, snapshot, restoreAcl),
+                tableDescriptor, snapshot, restoreAcl, customSFT),
         nonceKey);
       this.restoreTableToProcIdMap.put(tableName, procId);
       return procId;
@@ -822,7 +824,7 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
    * @throws IOException
    */
   public long restoreOrCloneSnapshot(final SnapshotDescription reqSnapshot, final NonceKey nonceKey,
-      final boolean restoreAcl) throws IOException {
+      final boolean restoreAcl, String customSFT) throws IOException {
     FileSystem fs = master.getMasterFileSystem().getFileSystem();
     Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(reqSnapshot, rootDir);
 
@@ -854,11 +856,12 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
     // Execute the restore/clone operation
     long procId;
     if (master.getTableDescriptors().exists(tableName)) {
-      procId = restoreSnapshot(reqSnapshot, tableName, snapshot, snapshotTableDesc, nonceKey,
-        restoreAcl);
+      procId =
+        restoreSnapshot(reqSnapshot, tableName, snapshot, snapshotTableDesc, nonceKey, restoreAcl);
     } else {
       procId =
-          cloneSnapshot(reqSnapshot, tableName, snapshot, snapshotTableDesc, nonceKey, restoreAcl);
+        cloneSnapshot(reqSnapshot, tableName, snapshot, snapshotTableDesc, nonceKey, restoreAcl,
+          customSFT);
     }
     return procId;
   }
@@ -880,6 +883,10 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
       final NonceKey nonceKey, final boolean restoreAcl) throws IOException {
     MasterCoprocessorHost cpHost = master.getMasterCoprocessorHost();
 
+    //have to check first if restoring the snapshot would break current SFT setup
+    StoreFileTrackerFactory.validatePreRestoreSnapshot(master.getTableDescriptors().get(tableName),
+      snapshotTableDesc, master.getConfiguration());
+
     if (master.getTableStateManager().isTableState(
       TableName.valueOf(snapshot.getTable()), TableState.State.ENABLED)) {
       throw new UnsupportedOperationException("Table '" +
@@ -921,7 +928,7 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
    * @return procId the ID of the restore snapshot procedure
    */
   private synchronized long restoreSnapshot(final SnapshotDescription snapshot,
-      final TableDescriptor tableDescriptor, final NonceKey nonceKey, final boolean restoreAcl)
+    final TableDescriptor tableDescriptor, final NonceKey nonceKey, final boolean restoreAcl)
       throws HBaseSnapshotException {
     final TableName tableName = tableDescriptor.getTableName();
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java
index 1c683ae..61a71c2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
 import org.apache.hadoop.hbase.regionserver.StoreContext;
 
 import org.apache.hadoop.hbase.regionserver.StoreUtils;
+import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
 import org.apache.hadoop.hbase.util.ReflectionUtils;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
@@ -92,7 +93,7 @@ public final class StoreFileTrackerFactory {
     return name != null ? name.name() : clazz.getName();
   }
 
-  private static Class<? extends StoreFileTracker> getTrackerClass(Configuration conf) {
+  public static Class<? extends StoreFileTracker> getTrackerClass(Configuration conf) {
     try {
       Trackers tracker = Trackers.valueOf(getStoreFileTrackerName(conf).toUpperCase());
       return tracker.clazz;
@@ -311,4 +312,40 @@ public final class StoreFileTrackerFactory {
       }
     }
   }
+
+  /**
+   * Makes sure restoring a snapshot does not break the current SFT setup
+   * follows StoreUtils.createStoreConfiguration
+   * @param currentTableDesc Existing Table's TableDescriptor
+   * @param snapshotTableDesc Snapshot's TableDescriptor
+   * @param baseConf Current global configuration
+   * @throws RestoreSnapshotException if restore would break the current SFT setup
+   */
+  public static void validatePreRestoreSnapshot(TableDescriptor currentTableDesc,
+    TableDescriptor snapshotTableDesc, Configuration baseConf) throws RestoreSnapshotException {
+
+    for (ColumnFamilyDescriptor cfDesc : currentTableDesc.getColumnFamilies()) {
+      ColumnFamilyDescriptor snapCFDesc = snapshotTableDesc.getColumnFamily(cfDesc.getName());
+      // if there is no counterpart in the snapshot it will be just deleted so the config does
+      // not matter
+      if (snapCFDesc != null) {
+        Configuration currentCompositeConf =
+          StoreUtils.createStoreConfiguration(baseConf, currentTableDesc, cfDesc);
+        Configuration snapCompositeConf =
+          StoreUtils.createStoreConfiguration(baseConf, snapshotTableDesc, snapCFDesc);
+        Class<? extends StoreFileTracker> currentSFT =
+          StoreFileTrackerFactory.getTrackerClass(currentCompositeConf);
+        Class<? extends StoreFileTracker> snapSFT =
+          StoreFileTrackerFactory.getTrackerClass(snapCompositeConf);
+
+        //restoration is not possible if there is an SFT mismatch
+        if (currentSFT != snapSFT) {
+          throw new RestoreSnapshotException(
+            "Restoring Snapshot is not possible because " + " the config for column family "
+              + cfDesc.getNameAsString() + " has incompatible configuration. Current SFT: "
+              + currentSFT + " SFT from snapshot: " + snapSFT);
+        }
+      }
+    }
+  }
 }
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 0f8a95f..1740644 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
@@ -57,6 +57,7 @@ import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
 import org.apache.hadoop.hbase.regionserver.StoreContext;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
+import org.apache.hadoop.hbase.regionserver.StoreUtils;
 import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTracker;
 import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory;
 import org.apache.hadoop.hbase.security.access.AccessControlClient;
@@ -72,9 +73,7 @@ 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;
@@ -200,8 +199,8 @@ public class RestoreSnapshotHelper {
 
     List<RegionInfo> tableRegions = getTableRegions();
 
-    RegionInfo mobRegion = MobUtils.getMobRegionInfo(snapshotManifest.getTableDescriptor()
-        .getTableName());
+    RegionInfo mobRegion =
+      MobUtils.getMobRegionInfo(snapshotManifest.getTableDescriptor().getTableName());
     if (tableRegions != null) {
       // restore the mob region in case
       if (regionNames.contains(mobRegion.getEncodedName())) {
@@ -707,7 +706,9 @@ public class RestoreSnapshotHelper {
           HRegionFileSystem.openRegionFromFileSystem(conf, fs, tableDir, newRegionInfo, false) :
           HRegionFileSystem.createRegionOnFileSystem(conf, fs, tableDir, newRegionInfo);
 
-        StoreFileTracker tracker = StoreFileTrackerFactory.create(conf, true,
+        Configuration sftConf = StoreUtils.createStoreConfiguration(conf, tableDesc,
+          tableDesc.getColumnFamily(familyFiles.getFamilyName().toByteArray()));
+        StoreFileTracker tracker = StoreFileTrackerFactory.create(sftConf, true,
           StoreContext.getBuilder().withFamilyStoreDirectoryPath(familyDir).
             withRegionFileSystem(regionFS).build());
         tracker.set(clonedFiles);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClientCustomSFT.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClientCustomSFT.java
new file mode 100644
index 0000000..9497967
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClientCustomSFT.java
@@ -0,0 +1,72 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThrows;
+
+import java.io.IOException;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory;
+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.EnvironmentEdgeManager;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ LargeTests.class, ClientTests.class })
+public class TestCloneSnapshotFromClientCustomSFT extends CloneSnapshotFromClientTestBase{
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestCloneSnapshotFromClientCustomSFT.class);
+
+  public static final String CLONE_SFT = "FILE";
+
+  @Test
+  public void testCloneSnapshotWithCustomSFT() throws IOException, InterruptedException {
+    TableName clonedTableName =
+      TableName.valueOf(getValidMethodName() + "-" + EnvironmentEdgeManager.currentTime());
+
+    admin.cloneSnapshot(Bytes.toString(snapshotName1), clonedTableName, false, CLONE_SFT);
+    verifyRowCount(TEST_UTIL, clonedTableName, snapshot1Rows);
+
+    TableDescriptor td = admin.getDescriptor(clonedTableName);
+    assertEquals(CLONE_SFT, td.getValue(StoreFileTrackerFactory.TRACKER_IMPL));
+
+    TEST_UTIL.deleteTable(clonedTableName);
+  }
+
+  @Test
+  public void testCloneSnapshotWithIncorrectCustomSFT() throws IOException, InterruptedException {
+    TableName clonedTableName =
+      TableName.valueOf(getValidMethodName() + "-" + EnvironmentEdgeManager.currentTime());
+
+    IOException ioException = assertThrows(IOException.class, () -> {
+      admin.cloneSnapshot(Bytes.toString(snapshotName1), clonedTableName, false, "IncorrectSFT");
+    });
+
+    assertEquals(
+      "java.lang.RuntimeException: java.lang.RuntimeException: " +
+        "java.lang.ClassNotFoundException: Class IncorrectSFT not found",
+      ioException.getMessage());
+  }
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTracker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTracker.java
index 9818972..fc54eb0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTracker.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTracker.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.regionserver.storefiletracker;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -28,7 +27,6 @@ import java.util.Map;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.regionserver.StoreContext;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
-import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTrackerFactory.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTrackerFactory.java
index 41f2afd..91038e9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTrackerFactory.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTrackerFactory.java
@@ -22,9 +22,16 @@ import static org.junit.Assert.assertThrows;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.regionserver.StoreContext;
+import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -55,4 +62,49 @@ public class TestStoreFileTrackerFactory {
     assertThrows(IllegalArgumentException.class, () -> StoreFileTrackerFactory
       .createForMigration(conf, configName, false, StoreContext.getBuilder().build()));
   }
+
+  @Test
+  public void testCheckSFTCompatibility() throws Exception {
+    //checking default value change on different configuration levels
+    Configuration conf = new Configuration();
+    conf.set(StoreFileTrackerFactory.TRACKER_IMPL, "DEFAULT");
+
+    //creating a TD with only TableDescriptor level config
+    TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(TableName.valueOf("TableX"));
+    builder.setValue(StoreFileTrackerFactory.TRACKER_IMPL, "FILE");
+    ColumnFamilyDescriptor cf = ColumnFamilyDescriptorBuilder.of("cf");
+    builder.setColumnFamily(cf);
+    TableDescriptor td = builder.build();
+
+    //creating a TD with matching ColumnFamilyDescriptor level setting
+    TableDescriptorBuilder snapBuilder =
+      TableDescriptorBuilder.newBuilder(TableName.valueOf("TableY"));
+    snapBuilder.setValue(StoreFileTrackerFactory.TRACKER_IMPL, "FILE");
+    ColumnFamilyDescriptorBuilder snapCFBuilder =
+      ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("cf"));
+    snapCFBuilder.setValue(StoreFileTrackerFactory.TRACKER_IMPL, "FILE");
+    snapBuilder.setColumnFamily(snapCFBuilder.build());
+    TableDescriptor snapTd = snapBuilder.build();
+
+    // adding a cf config that matches the td config is fine even when it does not match the default
+    StoreFileTrackerFactory.validatePreRestoreSnapshot(td, snapTd, conf);
+    // removing cf level config is fine when it matches the td config
+    StoreFileTrackerFactory.validatePreRestoreSnapshot(snapTd, td, conf);
+
+    TableDescriptorBuilder defaultBuilder =
+      TableDescriptorBuilder.newBuilder(TableName.valueOf("TableY"));
+    defaultBuilder.setValue(StoreFileTrackerFactory.TRACKER_IMPL, "FILE");
+    ColumnFamilyDescriptorBuilder defaultCFBuilder =
+      ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("cf"));
+    defaultCFBuilder.setValue(StoreFileTrackerFactory.TRACKER_IMPL, "DEFAULT");
+    defaultBuilder.setColumnFamily(defaultCFBuilder.build());
+    TableDescriptor defaultTd = defaultBuilder.build();
+
+    assertThrows(RestoreSnapshotException.class, () -> {
+      StoreFileTrackerFactory.validatePreRestoreSnapshot(td, defaultTd, conf);
+    });
+    assertThrows(RestoreSnapshotException.class, () -> {
+      StoreFileTrackerFactory.validatePreRestoreSnapshot(snapTd, defaultTd, conf);
+    });
+  }
 }
diff --git a/hbase-shell/src/main/ruby/hbase/admin.rb b/hbase-shell/src/main/ruby/hbase/admin.rb
index f4cfd31..6172f58 100644
--- a/hbase-shell/src/main/ruby/hbase/admin.rb
+++ b/hbase-shell/src/main/ruby/hbase/admin.rb
@@ -1203,8 +1203,8 @@ module Hbase
 
     #----------------------------------------------------------------------------------------------
     # Create a new table by cloning the snapshot content
-    def clone_snapshot(snapshot_name, table, restore_acl = false)
-      @admin.cloneSnapshot(snapshot_name, TableName.valueOf(table), restore_acl)
+    def clone_snapshot(snapshot_name, table, restore_acl = false, clone_sft = nil)
+      @admin.cloneSnapshot(snapshot_name, TableName.valueOf(table), restore_acl, clone_sft)
     end
 
     #----------------------------------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase_constants.rb b/hbase-shell/src/main/ruby/hbase_constants.rb
index 3c637b8..76631c9 100644
--- a/hbase-shell/src/main/ruby/hbase_constants.rb
+++ b/hbase-shell/src/main/ruby/hbase_constants.rb
@@ -40,6 +40,7 @@ module HBaseConstants
   CACHE = 'CACHE'.freeze
   CACHE_BLOCKS = 'CACHE_BLOCKS'.freeze
   CLASSNAME = 'CLASSNAME'.freeze
+  CLONE_SFT = 'CLONE_SFT'.freeze
   CLUSTER_KEY = 'CLUSTER_KEY'.freeze
   COLUMN = 'COLUMN'.freeze
   COLUMNS = 'COLUMNS'.freeze
diff --git a/hbase-shell/src/main/ruby/shell/commands/clone_snapshot.rb b/hbase-shell/src/main/ruby/shell/commands/clone_snapshot.rb
index abc97591..3edd16d 100644
--- a/hbase-shell/src/main/ruby/shell/commands/clone_snapshot.rb
+++ b/hbase-shell/src/main/ruby/shell/commands/clone_snapshot.rb
@@ -33,13 +33,17 @@ Following command will restore all acl from origin snapshot table into the
 newly created table.
 
   hbase> clone_snapshot 'snapshotName', 'namespace:tableName', {RESTORE_ACL=>true}
+
+StoreFileTracker implementation used after restore can be set by the following command.
+  hbase> clone_snapshot 'snapshotName', 'namespace:tableName', {CLONE_SFT=>'FILE'}
 EOF
       end
 
       def command(snapshot_name, table, args = {})
         raise(ArgumentError, 'Arguments should be a Hash') unless args.is_a?(Hash)
         restore_acl = args.delete(::HBaseConstants::RESTORE_ACL) || false
-        admin.clone_snapshot(snapshot_name, table, restore_acl)
+        clone_sft = args.delete(::HBaseConstants::CLONE_SFT) || nil
+        admin.clone_snapshot(snapshot_name, table, restore_acl, clone_sft)
       end
 
       def handle_exceptions(cause, *args)
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java
index 0aa9862..2a5d337 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java
@@ -1065,8 +1065,8 @@ public class ThriftAdmin implements Admin {
   }
 
   @Override
-  public Future<Void> cloneSnapshotAsync(String snapshotName, TableName tableName, boolean cloneAcl)
-      throws IOException, TableExistsException, RestoreSnapshotException {
+  public Future<Void> cloneSnapshotAsync(String snapshotName, TableName tableName, boolean cloneAcl,
+    String customSFT) throws IOException, TableExistsException, RestoreSnapshotException {
     throw new NotImplementedException("cloneSnapshotAsync not supported in ThriftAdmin");
   }
 

[hbase] 02/15: HBASE-25988 Store the store file list by a file (#3578)

Posted by zh...@apache.org.
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

commit 826e54ce78e77e6d1ae6e0cbb29dfb5cf9903ad4
Author: Duo Zhang <zh...@apache.org>
AuthorDate: Thu Aug 26 18:51:12 2021 +0800

    HBASE-25988 Store the store file list by a file (#3578)
    
    Signed-off-by: Wellington Chevreuil <wc...@apache.org>
---
 .../src/main/protobuf/StoreFileTracker.proto       |  29 +++--
 .../hadoop/hbase/regionserver/StoreContext.java    |   5 +
 .../hadoop/hbase/regionserver/StoreEngine.java     |   8 +-
 .../storefiletracker/DefaultStoreFileTracker.java  |   5 +-
 .../FileBasedStoreFileTracker.java                 | 142 +++++++++++++++++++++
 .../storefiletracker/StoreFileListFile.java        | 142 +++++++++++++++++++++
 .../storefiletracker/StoreFileTrackerBase.java     |  11 +-
 .../storefiletracker/StoreFileTrackerFactory.java  |  12 +-
 .../TestRegionWithFileBasedStoreFileTracker.java   | 109 ++++++++++++++++
 9 files changed, 430 insertions(+), 33 deletions(-)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java b/hbase-protocol-shaded/src/main/protobuf/StoreFileTracker.proto
similarity index 57%
copy from hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java
copy to hbase-protocol-shaded/src/main/protobuf/StoreFileTracker.proto
index 4f7231b..2a269ea 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java
+++ b/hbase-protocol-shaded/src/main/protobuf/StoreFileTracker.proto
@@ -15,21 +15,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hbase.regionserver.storefiletracker;
+syntax = "proto2";
+// This file contains protocol buffers that are used for store file tracker.
+package hbase.pb;
 
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.regionserver.StoreContext;
-import org.apache.yetus.audience.InterfaceAudience;
+option java_package = "org.apache.hadoop.hbase.shaded.protobuf.generated";
+option java_outer_classname = "StoreFileTrackerProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
 
-/**
- * Factory method for creating store file tracker.
- */
-@InterfaceAudience.Private
-public final class StoreFileTrackerFactory {
+message StoreFileEntry {
+  required string name = 1;
+  required uint64 size = 2;
+}
 
-  public static StoreFileTracker create(Configuration conf, TableName tableName,
-    boolean isPrimaryReplica, StoreContext ctx) {
-    return new DefaultStoreFileTracker(conf, tableName, isPrimaryReplica, ctx);
-  }
+message StoreFileList {
+  required uint64 timestamp = 1;
+  repeated StoreFileEntry store_file = 2;
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreContext.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreContext.java
index 2a9f968..588f8f4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreContext.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreContext.java
@@ -22,6 +22,7 @@ import java.util.Collection;
 import java.util.function.Supplier;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.io.HeapSize;
@@ -109,6 +110,10 @@ public final class StoreContext implements HeapSize {
     return coprocessorHost;
   }
 
+  public TableName getTableName() {
+    return getRegionInfo().getTable();
+  }
+
   public RegionInfo getRegionInfo() {
     return regionFileSystem.getRegionInfo();
   }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreEngine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreEngine.java
index 4033c33..0486729 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreEngine.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreEngine.java
@@ -173,9 +173,9 @@ public abstract class StoreEngine<SF extends StoreFlusher, CP extends Compaction
     return this.storeFlusher;
   }
 
-  private StoreFileTracker createStoreFileTracker(HStore store) {
-    return StoreFileTrackerFactory.create(store.conf, store.getRegionInfo().getTable(),
-      store.isPrimaryReplicaStore(), store.getStoreContext());
+  private StoreFileTracker createStoreFileTracker(Configuration conf, HStore store) {
+    return StoreFileTrackerFactory.create(conf, store.isPrimaryReplicaStore(),
+      store.getStoreContext());
   }
 
   /**
@@ -206,7 +206,7 @@ public abstract class StoreEngine<SF extends StoreFlusher, CP extends Compaction
     this.ctx = store.getStoreContext();
     this.coprocessorHost = store.getHRegion().getCoprocessorHost();
     this.openStoreFileThreadPoolCreator = store.getHRegion()::getStoreFileOpenAndCloseThreadPool;
-    this.storeFileTracker = createStoreFileTracker(store);
+    this.storeFileTracker = createStoreFileTracker(conf, store);
     assert compactor != null && compactionPolicy != null && storeFileManager != null &&
       storeFlusher != null && storeFileTracker != null;
   }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/DefaultStoreFileTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/DefaultStoreFileTracker.java
index d4c9a86..fa04481 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/DefaultStoreFileTracker.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/DefaultStoreFileTracker.java
@@ -21,7 +21,6 @@ import java.io.IOException;
 import java.util.Collection;
 import java.util.List;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.regionserver.StoreContext;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -33,9 +32,9 @@ import org.apache.yetus.audience.InterfaceAudience;
 @InterfaceAudience.Private
 class DefaultStoreFileTracker extends StoreFileTrackerBase {
 
-  public DefaultStoreFileTracker(Configuration conf, TableName tableName, boolean isPrimaryReplica,
+  public DefaultStoreFileTracker(Configuration conf, boolean isPrimaryReplica,
     StoreContext ctx) {
-    super(conf, tableName, isPrimaryReplica, ctx);
+    super(conf, isPrimaryReplica, ctx);
   }
 
   @Override
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/FileBasedStoreFileTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/FileBasedStoreFileTracker.java
new file mode 100644
index 0000000..de28b0e
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/FileBasedStoreFileTracker.java
@@ -0,0 +1,142 @@
+/**
+ * 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.regionserver.storefiletracker;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.regionserver.StoreContext;
+import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
+import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.StoreFileTrackerProtos.StoreFileEntry;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.StoreFileTrackerProtos.StoreFileList;
+
+/**
+ * A file based store file tracker.
+ * <p/>
+ * For this tracking way, the store file list will be persistent into a file, so we can write the
+ * new store files directly to the final data directory, as we will not load the broken files. This
+ * will greatly reduce the time for flush and compaction on some object storages as a rename is
+ * actual a copy on them. And it also avoid listing when loading store file list, which could also
+ * speed up the loading of store files as listing is also not a fast operation on most object
+ * storages.
+ */
+@InterfaceAudience.Private
+public class FileBasedStoreFileTracker extends StoreFileTrackerBase {
+
+  private final StoreFileListFile backedFile;
+
+  private final Map<String, StoreFileInfo> storefiles = new HashMap<>();
+
+  public FileBasedStoreFileTracker(Configuration conf, boolean isPrimaryReplica, StoreContext ctx) {
+    super(conf, isPrimaryReplica, ctx);
+    backedFile = new StoreFileListFile(ctx);
+  }
+
+  @Override
+  public List<StoreFileInfo> load() throws IOException {
+    StoreFileList list = backedFile.load();
+    if (list == null) {
+      return Collections.emptyList();
+    }
+    FileSystem fs = ctx.getRegionFileSystem().getFileSystem();
+    List<StoreFileInfo> infos = new ArrayList<>();
+    for (StoreFileEntry entry : list.getStoreFileList()) {
+      infos.add(ServerRegionReplicaUtil.getStoreFileInfo(conf, fs, ctx.getRegionInfo(),
+        ctx.getRegionFileSystem().getRegionInfoForFS(), ctx.getFamily().getNameAsString(),
+        new Path(ctx.getFamilyStoreDirectoryPath(), entry.getName())));
+    }
+    // In general, for primary replica, the load method should only be called once when
+    // initialization, so we do not need synchronized here. And for secondary replicas, though the
+    // load method could be called multiple times, we will never call other methods so no
+    // synchronized is also fine.
+    // But we have a refreshStoreFiles method in the Region interface, which can be called by CPs,
+    // and we have a RefreshHFilesEndpoint example to expose the refreshStoreFiles method as RPC, so
+    // for safety, let's still keep the synchronized here.
+    synchronized (storefiles) {
+      for (StoreFileInfo info : infos) {
+        storefiles.put(info.getPath().getName(), info);
+      }
+    }
+    return infos;
+  }
+
+  @Override
+  protected boolean requireWritingToTmpDirFirst() {
+    return false;
+  }
+
+  private StoreFileEntry toStoreFileEntry(StoreFileInfo info) {
+    return StoreFileEntry.newBuilder().setName(info.getPath().getName()).setSize(info.getSize())
+      .build();
+  }
+
+  @Override
+  protected void doAddNewStoreFiles(Collection<StoreFileInfo> newFiles) throws IOException {
+    synchronized (storefiles) {
+      StoreFileList.Builder builder = StoreFileList.newBuilder();
+      for (StoreFileInfo info : storefiles.values()) {
+        builder.addStoreFile(toStoreFileEntry(info));
+      }
+      for (StoreFileInfo info : newFiles) {
+        builder.addStoreFile(toStoreFileEntry(info));
+      }
+      backedFile.update(builder);
+      for (StoreFileInfo info : newFiles) {
+        storefiles.put(info.getPath().getName(), info);
+      }
+    }
+  }
+
+  @Override
+  protected void doAddCompactionResults(Collection<StoreFileInfo> compactedFiles,
+    Collection<StoreFileInfo> newFiles) throws IOException {
+    Set<String> compactedFileNames =
+      compactedFiles.stream().map(info -> info.getPath().getName()).collect(Collectors.toSet());
+    synchronized (storefiles) {
+      StoreFileList.Builder builder = StoreFileList.newBuilder();
+      storefiles.forEach((name, info) -> {
+        if (compactedFileNames.contains(name)) {
+          return;
+        }
+        builder.addStoreFile(toStoreFileEntry(info));
+      });
+      for (StoreFileInfo info : newFiles) {
+        builder.addStoreFile(toStoreFileEntry(info));
+      }
+      backedFile.update(builder);
+      for (String name : compactedFileNames) {
+        storefiles.remove(name);
+      }
+      for (StoreFileInfo info : newFiles) {
+        storefiles.put(info.getPath().getName(), info);
+      }
+    }
+  }
+}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileListFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileListFile.java
new file mode 100644
index 0000000..c778bfc
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileListFile.java
@@ -0,0 +1,142 @@
+/**
+ * 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.regionserver.storefiletracker;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.regionserver.StoreContext;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+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.hbase.thirdparty.com.google.common.io.ByteStreams;
+import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.StoreFileTrackerProtos.StoreFileList;
+
+/**
+ * To fully avoid listing, here we use two files for tracking. When loading, we will try to read
+ * both the two files, if only one exists, we will trust this one, if both exist, we will compare
+ * the timestamp to see which one is newer and trust that one. And we will record in memory that
+ * which one is trusted by us, and when we need to update the store file list, we will write to the
+ * other file.
+ * <p/>
+ * So in this way, we could avoid listing when we want to load the store file list file.
+ */
+@InterfaceAudience.Private
+class StoreFileListFile {
+
+  private static final Logger LOG = LoggerFactory.getLogger(StoreFileListFile.class);
+
+  private static final String TRACK_FILE_DIR = ".filelist";
+
+  private static final String TRACK_FILE = "f1";
+
+  private static final String TRACK_FILE_ROTATE = "f2";
+
+  private final StoreContext ctx;
+
+  private final Path trackFileDir;
+
+  private final Path[] trackFiles = new Path[2];
+
+  // this is used to make sure that we do not go backwards
+  private long prevTimestamp = -1;
+
+  private int nextTrackFile = -1;
+
+  StoreFileListFile(StoreContext ctx) {
+    this.ctx = ctx;
+    trackFileDir = new Path(ctx.getFamilyStoreDirectoryPath(), TRACK_FILE_DIR);
+    trackFiles[0] = new Path(trackFileDir, TRACK_FILE);
+    trackFiles[1] = new Path(trackFileDir, TRACK_FILE_ROTATE);
+  }
+
+  private StoreFileList load(Path path) throws IOException {
+    FileSystem fs = ctx.getRegionFileSystem().getFileSystem();
+    byte[] bytes;
+    try (FSDataInputStream in = fs.open(path)) {
+      bytes = ByteStreams.toByteArray(in);
+    }
+    // Read all the bytes and then parse it, so we will only throw InvalidProtocolBufferException
+    // here. This is very important for upper layer to determine whether this is the normal case,
+    // where the file does not exist or is incomplete. If there is another type of exception, the
+    // upper layer should throw it out instead of just ignoring it, otherwise it will lead to data
+    // loss.
+    return StoreFileList.parseFrom(bytes);
+  }
+
+  private int select(StoreFileList[] lists) {
+    if (lists[0] == null) {
+      return 1;
+    }
+    if (lists[1] == null) {
+      return 0;
+    }
+    return lists[0].getTimestamp() >= lists[1].getTimestamp() ? 0 : 1;
+  }
+
+  StoreFileList load() throws IOException {
+    StoreFileList[] lists = new StoreFileList[2];
+    for (int i = 0; i < 2; i++) {
+      try {
+        lists[i] = load(trackFiles[i]);
+      } catch (FileNotFoundException | InvalidProtocolBufferException e) {
+        // this is normal case, so use info and do not log stacktrace
+        LOG.info("Failed to load track file {}: {}", trackFiles[i], e);
+      }
+    }
+    int winnerIndex = select(lists);
+    if (lists[winnerIndex] != null) {
+      nextTrackFile = 1 - winnerIndex;
+      prevTimestamp = lists[winnerIndex].getTimestamp();
+    } else {
+      nextTrackFile = 0;
+    }
+    return lists[winnerIndex];
+  }
+
+  /**
+   * We will set the timestamp in this method so just pass the builder in
+   */
+  void update(StoreFileList.Builder builder) throws IOException {
+    Preconditions.checkState(nextTrackFile >= 0, "should call load first before calling update");
+    FileSystem fs = ctx.getRegionFileSystem().getFileSystem();
+    long timestamp = Math.max(prevTimestamp + 1, EnvironmentEdgeManager.currentTime());
+    try (FSDataOutputStream out = fs.create(trackFiles[nextTrackFile], true)) {
+      builder.setTimestamp(timestamp).build().writeTo(out);
+    }
+    // record timestamp
+    prevTimestamp = timestamp;
+    // rotate the file
+    nextTrackFile = 1 - nextTrackFile;
+    try {
+      fs.delete(trackFiles[nextTrackFile], false);
+    } catch (IOException e) {
+      // we will create new file with overwrite = true, so not a big deal here, only for speed up
+      // loading as we do not need to read this file when loading(we will hit FileNotFoundException)
+      LOG.debug("failed to delete old track file {}, not a big deal, just ignore", e);
+    }
+  }
+}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java
index 2451f45..92c6992 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java
@@ -21,7 +21,6 @@ import java.io.IOException;
 import java.util.Collection;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.crypto.Encryption;
@@ -51,18 +50,14 @@ abstract class StoreFileTrackerBase implements StoreFileTracker {
 
   protected final Configuration conf;
 
-  protected final TableName tableName;
-
   protected final boolean isPrimaryReplica;
 
   protected final StoreContext ctx;
 
   private volatile boolean cacheOnWriteLogged;
 
-  protected StoreFileTrackerBase(Configuration conf, TableName tableName, boolean isPrimaryReplica,
-    StoreContext ctx) {
+  protected StoreFileTrackerBase(Configuration conf, boolean isPrimaryReplica, StoreContext ctx) {
     this.conf = conf;
-    this.tableName = tableName;
     this.isPrimaryReplica = isPrimaryReplica;
     this.ctx = ctx;
   }
@@ -95,7 +90,7 @@ abstract class StoreFileTrackerBase implements StoreFileTracker {
       .withBlockSize(family.getBlocksize()).withHBaseCheckSum(true)
       .withDataBlockEncoding(family.getDataBlockEncoding()).withEncryptionContext(encryptionContext)
       .withCreateTime(EnvironmentEdgeManager.currentTime()).withColumnFamily(family.getName())
-      .withTableName(tableName.getName()).withCellComparator(ctx.getComparator()).build();
+      .withTableName(ctx.getTableName().getName()).withCellComparator(ctx.getComparator()).build();
     return hFileContext;
   }
 
@@ -153,7 +148,7 @@ abstract class StoreFileTrackerBase implements StoreFileTracker {
       outputDir =
         new Path(ctx.getRegionFileSystem().getTempDir(), ctx.getFamily().getNameAsString());
     } else {
-      throw new UnsupportedOperationException("not supported yet");
+      outputDir = ctx.getFamilyStoreDirectoryPath();
     }
     StoreFileWriter.Builder builder =
       new StoreFileWriter.Builder(conf, writerCacheConf, ctx.getRegionFileSystem().getFileSystem())
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java
index 4f7231b..6cdfaf4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java
@@ -18,8 +18,8 @@
 package org.apache.hadoop.hbase.regionserver.storefiletracker;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.regionserver.StoreContext;
+import org.apache.hadoop.hbase.util.ReflectionUtils;
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
@@ -28,8 +28,12 @@ import org.apache.yetus.audience.InterfaceAudience;
 @InterfaceAudience.Private
 public final class StoreFileTrackerFactory {
 
-  public static StoreFileTracker create(Configuration conf, TableName tableName,
-    boolean isPrimaryReplica, StoreContext ctx) {
-    return new DefaultStoreFileTracker(conf, tableName, isPrimaryReplica, ctx);
+  public static final String TRACK_IMPL = "hbase.store.file-tracker.impl";
+
+  public static StoreFileTracker create(Configuration conf, boolean isPrimaryReplica,
+    StoreContext ctx) {
+    Class<? extends StoreFileTracker> tracker =
+      conf.getClass(TRACK_IMPL, DefaultStoreFileTracker.class, StoreFileTracker.class);
+    return ReflectionUtils.newInstance(tracker, conf, isPrimaryReplica, ctx);
   }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestRegionWithFileBasedStoreFileTracker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestRegionWithFileBasedStoreFileTracker.java
new file mode 100644
index 0000000..a434934
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestRegionWithFileBasedStoreFileTracker.java
@@ -0,0 +1,109 @@
+/**
+ * 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.regionserver.storefiletracker;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+@Category({ RegionServerTests.class, MediumTests.class })
+public class TestRegionWithFileBasedStoreFileTracker {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestRegionWithFileBasedStoreFileTracker.class);
+
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  private static final byte[] CF = Bytes.toBytes("cf");
+
+  private static final byte[] CQ = Bytes.toBytes("cq");
+
+  private static final TableDescriptor TD =
+    TableDescriptorBuilder.newBuilder(TableName.valueOf("file_based_tracker"))
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(CF)).build();
+
+  private static final RegionInfo RI = RegionInfoBuilder.newBuilder(TD.getTableName()).build();
+
+  @Rule
+  public TestName name = new TestName();
+
+  private HRegion region;
+
+  @Before
+  public void setUp() throws IOException {
+    Configuration conf = new Configuration(UTIL.getConfiguration());
+    conf.setClass(StoreFileTrackerFactory.TRACK_IMPL, FileBasedStoreFileTracker.class,
+      StoreFileTracker.class);
+    region = HBaseTestingUtility.createRegionAndWAL(RI, UTIL.getDataTestDir(name.getMethodName()),
+      conf, TD);
+  }
+
+  @After
+  public void tearDown() throws IOException {
+    if (region != null) {
+      HBaseTestingUtility.closeRegionAndWAL(region);
+    }
+    UTIL.cleanupTestDir();
+  }
+
+  @Test
+  public void testFlushAndCompaction() throws IOException {
+    for (int i = 0; i < 10; i++) {
+      for (int j = 0; j < 10; j++) {
+        int v = i * 10 + j;
+        region.put(new Put(Bytes.toBytes(v)).addColumn(CF, CQ, Bytes.toBytes(v)));
+      }
+      region.flush(true);
+      if (i % 3 == 2) {
+        region.compact(true);
+      }
+    }
+    // reopen the region, make sure the store file tracker works, i.e, we can get all the records
+    // back
+    region.close();
+    region = HRegion.openHRegion(region, null);
+    for (int i = 0; i < 100; i++) {
+      Result result = region.get(new Get(Bytes.toBytes(i)));
+      assertEquals(i, Bytes.toInt(result.getValue(CF, CQ)));
+    }
+  }
+}

[hbase] 12/15: HBASE-26263 [Rolling Upgrading] Persist the StoreFileTracker configurations to TableDescriptor for existing tables (#3700)

Posted by zh...@apache.org.
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

commit c4325ff0889bafba22de71a31a4c5b5c6567f409
Author: GeorryHuang <hu...@apache.org>
AuthorDate: Sat Nov 6 22:20:12 2021 +0800

    HBASE-26263 [Rolling Upgrading] Persist the StoreFileTracker configurations to TableDescriptor for existing tables (#3700)
    
    Signed-off-by: Duo Zhang <zh...@apache.org>
    Reviewed-by: Wellington Ramos Chevreuil <wc...@apache.org>
---
 .../hbase/client/TableDescriptorBuilder.java       |  16 ++
 .../src/main/protobuf/MasterProcedure.proto        |  11 ++
 .../org/apache/hadoop/hbase/master/HMaster.java    |   6 +
 .../hbase/master/migrate/RollingUpgradeChore.java  | 130 +++++++++++++++++
 .../procedure/ModifyTableDescriptorProcedure.java  | 161 +++++++++++++++++++++
 .../MigrateStoreFileTrackerProcedure.java          |  48 ++++++
 .../migrate/TestMigrateStoreFileTracker.java       | 108 ++++++++++++++
 7 files changed, 480 insertions(+)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java
index 823b61e..d9a1515 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java
@@ -440,6 +440,11 @@ public class TableDescriptorBuilder {
     return this;
   }
 
+  public TableDescriptorBuilder removeValue(final String key) {
+    desc.removeValue(key);
+    return this;
+  }
+
   public TableDescriptorBuilder removeValue(Bytes key) {
     desc.removeValue(key);
     return this;
@@ -795,6 +800,17 @@ public class TableDescriptorBuilder {
      * parameters.
      * @return the modifyable TD
      */
+    public ModifyableTableDescriptor removeValue(final String key) {
+      return setValue(key, (String) null);
+    }
+
+    /**
+     * Remove metadata represented by the key from the {@link #values} map
+     *
+     * @param key Key whose key and value we're to remove from TableDescriptor
+     * parameters.
+     * @return the modifyable TD
+     */
     public ModifyableTableDescriptor removeValue(Bytes key) {
       return setValue(key, (Bytes) null);
     }
diff --git a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
index fc97290..66d1640 100644
--- a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
@@ -584,3 +584,14 @@ enum ClaimReplicationQueuesState {
   CLAIM_REPLICATION_QUEUES_DISPATCH = 1;
   CLAIM_REPLICATION_QUEUES_FINISH = 2;
 }
+
+
+enum ModifyTableDescriptorState {
+  MODIFY_TABLE_DESCRIPTOR_PREPARE = 1;
+  MODIFY_TABLE_DESCRIPTOR_UPDATE = 2;
+}
+
+message ModifyTableDescriptorStateData {
+  required TableSchema unmodified_table_schema = 1;
+  optional TableSchema modified_table_schema = 2;
+}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index e205c8d..e67ab53 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -130,6 +130,7 @@ import org.apache.hadoop.hbase.master.http.MasterRedirectServlet;
 import org.apache.hadoop.hbase.master.http.MasterStatusServlet;
 import org.apache.hadoop.hbase.master.janitor.CatalogJanitor;
 import org.apache.hadoop.hbase.master.locking.LockManager;
+import org.apache.hadoop.hbase.master.migrate.RollingUpgradeChore;
 import org.apache.hadoop.hbase.master.normalizer.RegionNormalizerFactory;
 import org.apache.hadoop.hbase.master.normalizer.RegionNormalizerManager;
 import org.apache.hadoop.hbase.master.procedure.CreateTableProcedure;
@@ -355,6 +356,7 @@ public class HMaster extends HRegionServer implements MasterServices {
   private ExpiredMobFileCleanerChore expiredMobFileCleanerChore;
   private MobCompactionChore mobCompactChore;
   private MasterMobCompactionThread mobCompactThread;
+  private RollingUpgradeChore rollingUpgradeChore;
   // used to synchronize the mobCompactionStates
   private final IdLock mobCompactionLock = new IdLock();
   // save the information of mob compactions in tables.
@@ -1212,6 +1214,9 @@ public class HMaster extends HRegionServer implements MasterServices {
       LOG.debug("Balancer post startup initialization complete, took " + (
           (EnvironmentEdgeManager.currentTime() - start) / 1000) + " seconds");
     }
+
+    this.rollingUpgradeChore = new RollingUpgradeChore(this);
+    getChoreService().scheduleChore(rollingUpgradeChore);
   }
 
   private void createMissingCFsInMetaDuringUpgrade(
@@ -1695,6 +1700,7 @@ public class HMaster extends HRegionServer implements MasterServices {
       shutdownChore(snapshotCleanerChore);
       shutdownChore(hbckChore);
       shutdownChore(regionsRecoveryChore);
+      shutdownChore(rollingUpgradeChore);
     }
   }
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/migrate/RollingUpgradeChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/migrate/RollingUpgradeChore.java
new file mode 100644
index 0000000..3896b41
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/migrate/RollingUpgradeChore.java
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.master.migrate;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.TableDescriptors;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
+import org.apache.hadoop.hbase.regionserver.storefiletracker.MigrateStoreFileTrackerProcedure;
+import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * To avoid too many migrating/upgrade threads to be submitted at the time during master
+ * initialization, RollingUpgradeChore handles all rolling-upgrade tasks.
+ * */
+@InterfaceAudience.Private
+public class RollingUpgradeChore extends ScheduledChore {
+
+  static final String ROLLING_UPGRADE_CHORE_PERIOD_SECONDS_KEY =
+    "hbase.master.rolling.upgrade.chore.period.secs";
+  static final int DFAULT_ROLLING_UPGRADE_CHORE_PERIOD_SECONDS = 10; // 10 seconds by default
+
+  static final String ROLLING_UPGRADE_CHORE_DELAY_SECONDS_KEY =
+    "hbase.master.rolling.upgrade.chore.delay.secs";
+  static final long DEFAULT_ROLLING_UPGRADE_CHORE_DELAY_SECONDS = 30; // 30 seconds
+
+  static final int CONCURRENT_PROCEDURES_COUNT = 5;
+
+  private final static Logger LOG = LoggerFactory.getLogger(RollingUpgradeChore.class);
+  ProcedureExecutor<MasterProcedureEnv> procedureExecutor;
+  private TableDescriptors tableDescriptors;
+  private List<MigrateStoreFileTrackerProcedure> processingProcs = new ArrayList<>();
+
+  public RollingUpgradeChore(MasterServices masterServices) {
+    this(masterServices.getConfiguration(), masterServices.getMasterProcedureExecutor(),
+      masterServices.getTableDescriptors(), masterServices);
+  }
+
+  private RollingUpgradeChore(Configuration conf,
+    ProcedureExecutor<MasterProcedureEnv> procedureExecutor, TableDescriptors tableDescriptors,
+    Stoppable stopper) {
+    super(RollingUpgradeChore.class.getSimpleName(), stopper, conf
+        .getInt(ROLLING_UPGRADE_CHORE_PERIOD_SECONDS_KEY,
+          DFAULT_ROLLING_UPGRADE_CHORE_PERIOD_SECONDS), conf
+        .getLong(ROLLING_UPGRADE_CHORE_DELAY_SECONDS_KEY,
+          DEFAULT_ROLLING_UPGRADE_CHORE_DELAY_SECONDS),
+      TimeUnit.SECONDS);
+    this.procedureExecutor = procedureExecutor;
+    this.tableDescriptors = tableDescriptors;
+  }
+
+  @Override
+  protected void chore() {
+    if (isCompletelyMigrateSFT(CONCURRENT_PROCEDURES_COUNT)) {
+      LOG.info("All Rolling-Upgrade tasks are complete, shutdown RollingUpgradeChore!");
+      shutdown();
+    }
+  }
+
+  private boolean isCompletelyMigrateSFT(int concurrentCount){
+    Iterator<MigrateStoreFileTrackerProcedure> iter = processingProcs.iterator();
+    while(iter.hasNext()){
+      MigrateStoreFileTrackerProcedure proc = iter.next();
+      if(procedureExecutor.isFinished(proc.getProcId())){
+        iter.remove();
+      }
+    }
+    // No new migration procedures will be submitted until
+    // all procedures executed last time are completed.
+    if (!processingProcs.isEmpty()) {
+      return false;
+    }
+
+    Map<String, TableDescriptor> migrateSFTTables;
+    try {
+      migrateSFTTables = tableDescriptors.getAll().entrySet().stream().filter(entry -> {
+        TableDescriptor td = entry.getValue();
+        return StringUtils.isEmpty(td.getValue(StoreFileTrackerFactory.TRACKER_IMPL));
+      }).limit(concurrentCount).collect(Collectors.toMap(e -> e.getKey(), e -> e.getValue()));
+    } catch (IOException e) {
+      LOG.warn("Failed to migrate StoreFileTracker", e);
+      return false;
+    }
+
+    if (migrateSFTTables.isEmpty()) {
+      LOG.info("There is no table to migrate StoreFileTracker!");
+      return true;
+    }
+
+    for (Map.Entry<String, TableDescriptor> entry : migrateSFTTables.entrySet()) {
+      TableDescriptor tableDescriptor = entry.getValue();
+      MigrateStoreFileTrackerProcedure proc =
+        new MigrateStoreFileTrackerProcedure(procedureExecutor.getEnvironment(), tableDescriptor);
+      procedureExecutor.submitProcedure(proc);
+      processingProcs.add(proc);
+    }
+    return false;
+  }
+}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableDescriptorProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableDescriptorProcedure.java
new file mode 100644
index 0000000..4f49142
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableDescriptorProcedure.java
@@ -0,0 +1,161 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.procedure;
+
+import java.io.IOException;
+import java.util.Optional;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
+import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
+import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
+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.ModifyTableDescriptorState;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyTableDescriptorStateData;
+
+/**
+ * The procedure will only update the table descriptor without reopening all the regions.
+ * <p/>
+ * It is usually used for migrating when upgrading, where we need to add something into the table
+ * descriptor, such as the rs group information.
+ */
+@InterfaceAudience.Private
+public abstract class ModifyTableDescriptorProcedure
+  extends AbstractStateMachineTableProcedure<ModifyTableDescriptorState> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ModifyTableDescriptorProcedure.class);
+
+  private TableDescriptor unmodifiedTableDescriptor;
+  private TableDescriptor modifiedTableDescriptor;
+
+  protected ModifyTableDescriptorProcedure() {
+  }
+
+  protected ModifyTableDescriptorProcedure(MasterProcedureEnv env, TableDescriptor unmodified) {
+    super(env);
+    this.unmodifiedTableDescriptor = unmodified;
+  }
+
+  @Override
+  public TableName getTableName() {
+    return unmodifiedTableDescriptor.getTableName();
+  }
+
+  @Override
+  public TableOperationType getTableOperationType() {
+    return TableOperationType.EDIT;
+  }
+
+  /**
+   * Sub class should implement this method to modify the table descriptor, such as storing the rs
+   * group information.
+   * <p/>
+   * Since the migrating is asynchronouns, it is possible that users have already changed the rs
+   * group for a table, in this case we do not need to modify the table descriptor any more, then
+   * you could just return {@link Optional#empty()}.
+   */
+  protected abstract Optional<TableDescriptor> modify(MasterProcedureEnv env,
+    TableDescriptor current) throws IOException;
+
+  @Override
+  protected Flow executeFromState(MasterProcedureEnv env, ModifyTableDescriptorState state)
+    throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException {
+    try {
+      switch (state) {
+        case MODIFY_TABLE_DESCRIPTOR_PREPARE:
+          Optional<TableDescriptor> modified = modify(env, unmodifiedTableDescriptor);
+          if (modified.isPresent()) {
+            modifiedTableDescriptor = modified.get();
+            setNextState(ModifyTableDescriptorState.MODIFY_TABLE_DESCRIPTOR_UPDATE);
+            return Flow.HAS_MORE_STATE;
+          } else {
+            // do not need to modify
+            return Flow.NO_MORE_STATE;
+          }
+        case MODIFY_TABLE_DESCRIPTOR_UPDATE:
+          env.getMasterServices().getTableDescriptors().update(modifiedTableDescriptor);
+          return Flow.NO_MORE_STATE;
+        default:
+          throw new UnsupportedOperationException("unhandled state=" + state);
+      }
+    } catch (IOException e) {
+      if (isRollbackSupported(state)) {
+        setFailure("master-modify-table-descriptor", e);
+      } else {
+        LOG.warn("Retriable error trying to modify table descriptor={} (in state={})",
+          getTableName(), state, e);
+      }
+    }
+    return Flow.HAS_MORE_STATE;
+  }
+
+  @Override
+  protected void rollbackState(MasterProcedureEnv env, ModifyTableDescriptorState state)
+    throws IOException, InterruptedException {
+    if (state == ModifyTableDescriptorState.MODIFY_TABLE_DESCRIPTOR_PREPARE) {
+      return;
+    }
+    throw new UnsupportedOperationException("unhandled state=" + state);
+  }
+
+  @Override
+  protected boolean isRollbackSupported(ModifyTableDescriptorState state) {
+    return state == ModifyTableDescriptorState.MODIFY_TABLE_DESCRIPTOR_PREPARE;
+  }
+
+  @Override
+  protected ModifyTableDescriptorState getState(int stateId) {
+    return ModifyTableDescriptorState.forNumber(stateId);
+  }
+
+  @Override
+  protected int getStateId(ModifyTableDescriptorState state) {
+    return state.getNumber();
+  }
+
+  @Override
+  protected ModifyTableDescriptorState getInitialState() {
+    return ModifyTableDescriptorState.MODIFY_TABLE_DESCRIPTOR_PREPARE;
+  }
+
+  @Override
+  protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException {
+    super.serializeStateData(serializer);
+    ModifyTableDescriptorStateData.Builder builder = ModifyTableDescriptorStateData.newBuilder()
+      .setUnmodifiedTableSchema(ProtobufUtil.toTableSchema(unmodifiedTableDescriptor));
+    if (modifiedTableDescriptor != null) {
+      builder.setModifiedTableSchema(ProtobufUtil.toTableSchema(modifiedTableDescriptor));
+    }
+    serializer.serialize(builder.build());
+  }
+
+  @Override
+  protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException {
+    super.deserializeStateData(serializer);
+    ModifyTableDescriptorStateData data =
+      serializer.deserialize(ModifyTableDescriptorStateData.class);
+    unmodifiedTableDescriptor = ProtobufUtil.toTableDescriptor(data.getUnmodifiedTableSchema());
+    if (data.hasModifiedTableSchema()) {
+      modifiedTableDescriptor = ProtobufUtil.toTableDescriptor(data.getModifiedTableSchema());
+    }
+  }
+}
\ No newline at end of file
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrateStoreFileTrackerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrateStoreFileTrackerProcedure.java
new file mode 100644
index 0000000..7cf3d1e
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrateStoreFileTrackerProcedure.java
@@ -0,0 +1,48 @@
+/**
+ * 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.regionserver.storefiletracker;
+
+import java.util.Optional;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.master.procedure.ModifyTableDescriptorProcedure;
+import org.apache.hadoop.hbase.procedure2.util.StringUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Procedure for migrating StoreFileTracker information to table descriptor.
+ */
+@InterfaceAudience.Private
+public class MigrateStoreFileTrackerProcedure extends ModifyTableDescriptorProcedure {
+
+  public MigrateStoreFileTrackerProcedure(){}
+
+  public MigrateStoreFileTrackerProcedure(MasterProcedureEnv env, TableDescriptor unmodified) {
+    super(env, unmodified);
+  }
+
+  @Override
+  protected Optional<TableDescriptor> modify(MasterProcedureEnv env, TableDescriptor current) {
+    if (StringUtils.isEmpty(current.getValue(StoreFileTrackerFactory.TRACKER_IMPL))) {
+      TableDescriptor td =
+        StoreFileTrackerFactory.updateWithTrackerConfigs(env.getMasterConfiguration(), current);
+      return Optional.of(td);
+    }
+    return Optional.empty();
+  }
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/migrate/TestMigrateStoreFileTracker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/migrate/TestMigrateStoreFileTracker.java
new file mode 100644
index 0000000..4f00184
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/migrate/TestMigrateStoreFileTracker.java
@@ -0,0 +1,108 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.migrate;
+
+import java.io.IOException;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableDescriptors;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(MediumTests.class)
+public class TestMigrateStoreFileTracker {
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestMigrateStoreFileTracker.class);
+  private final static String[] tables = new String[] { "t1", "t2", "t3", "t4", "t5", "t6" };
+  private final static String famStr = "f1";
+  private final static byte[] fam = Bytes.toBytes(famStr);
+
+  private HBaseTestingUtility HTU;
+  private Configuration conf;
+  private HTableDescriptor tableDescriptor;
+
+  @Before
+  public void setUp() throws Exception {
+    conf = HBaseConfiguration.create();
+    //Speed up the launch of RollingUpgradeChore
+    conf.setInt(RollingUpgradeChore.ROLLING_UPGRADE_CHORE_PERIOD_SECONDS_KEY, 1);
+    conf.setLong(RollingUpgradeChore.ROLLING_UPGRADE_CHORE_DELAY_SECONDS_KEY, 1);
+    HTU = new HBaseTestingUtility(conf);
+    HTU.startMiniCluster();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    HTU.shutdownMiniCluster();
+  }
+
+  @Test
+  public void testMigrateStoreFileTracker() throws IOException, InterruptedException {
+    //create tables to test
+    for (int i = 0; i < tables.length; i++) {
+      tableDescriptor = HTU.createTableDescriptor(tables[i]);
+      tableDescriptor.addFamily(new HColumnDescriptor(fam));
+      HTU.createTable(tableDescriptor, null);
+    }
+    TableDescriptors tableDescriptors = HTU.getMiniHBaseCluster().getMaster().getTableDescriptors();
+    for (int i = 0; i < tables.length; i++) {
+      TableDescriptor tdAfterCreated = tableDescriptors.get(TableName.valueOf(tables[i]));
+      //make sure that TRACKER_IMPL was set by default after tables have been created.
+      Assert.assertNotNull(tdAfterCreated.getValue(StoreFileTrackerFactory.TRACKER_IMPL));
+      //Remove StoreFileTracker impl from tableDescriptor
+      TableDescriptor tdRemovedSFT = TableDescriptorBuilder.newBuilder(tdAfterCreated)
+        .removeValue(StoreFileTrackerFactory.TRACKER_IMPL).build();
+      tableDescriptors.update(tdRemovedSFT);
+    }
+    HTU.getMiniHBaseCluster().stopMaster(0).join();
+    HTU.getMiniHBaseCluster().startMaster();
+    HTU.getMiniHBaseCluster().waitForActiveAndReadyMaster(30000);
+    //wait until all tables have been migrated
+    TableDescriptors tds = HTU.getMiniHBaseCluster().getMaster().getTableDescriptors();
+    HTU.waitFor(30000, () -> {
+      try {
+        for (int i = 0; i < tables.length; i++) {
+          TableDescriptor td = tds.get(TableName.valueOf(tables[i]));
+          if (StringUtils.isEmpty(td.getValue(StoreFileTrackerFactory.TRACKER_IMPL))) {
+            return false;
+          }
+        }
+        return true;
+      } catch (IOException e) {
+        return false;
+      }
+    });
+  }
+}

[hbase] 05/15: HBASE-26246 Persist the StoreFileTracker configurations to TableDescriptor when creating table (#3666)

Posted by zh...@apache.org.
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

commit 26f263b79277b574af550b7288ae050f4551dae1
Author: Wellington Ramos Chevreuil <wc...@apache.org>
AuthorDate: Sun Sep 12 14:14:03 2021 +0100

    HBASE-26246 Persist the StoreFileTracker configurations to TableDescriptor when creating table (#3666)
    
    Signed-off-by: Duo Zhang <zh...@apache.org>
---
 .../hbase/client/TableDescriptorBuilder.java       |  4 ++++
 .../master/procedure/CreateTableProcedure.java     |  7 ++++++
 .../hbase/regionserver/HRegionFileSystem.java      |  2 +-
 .../MigrationStoreFileTracker.java                 |  8 +++++++
 .../storefiletracker/StoreFileTracker.java         |  8 +++++++
 .../storefiletracker/StoreFileTrackerBase.java     | 13 +++++++++++
 .../storefiletracker/StoreFileTrackerFactory.java  | 25 +++++++++++++++-------
 .../org/apache/hadoop/hbase/client/TestAdmin.java  |  6 ++++++
 .../org/apache/hadoop/hbase/client/TestAdmin3.java |  6 ++++++
 .../hbase/client/TestAsyncTableAdminApi.java       |  7 ++++++
 .../hbase/client/TestAsyncTableAdminApi3.java      |  8 ++++++-
 .../procedure/MasterProcedureTestingUtility.java   |  7 ++++++
 .../master/procedure/TestCreateTableProcedure.java | 17 +++++++++++++++
 .../TestMigrationStoreFileTracker.java             |  6 +++---
 .../storefiletracker/TestStoreFileTracker.java     | 14 +++++++++---
 15 files changed, 122 insertions(+), 16 deletions(-)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java
index c2ca5a1..823b61e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java
@@ -581,6 +581,10 @@ public class TableDescriptorBuilder {
     return this;
   }
 
+  public String getValue(String key) {
+    return desc.getValue(key);
+  }
+
   /**
    * Sets replication scope all & only the columns already in the builder. Columns added later won't
    * be backfilled with replication scope.
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 4242ed6..219821f 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
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hbase.master.procedure;
 
+
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
@@ -31,10 +32,12 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
 import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 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.regionserver.storefiletracker.StoreFileTrackerFactory;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.FSTableDescriptors;
@@ -265,6 +268,10 @@ public class CreateTableProcedure
           getTableName(), (newRegions != null ? newRegions.size() : 0));
     }
 
+    TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableDescriptor);
+    StoreFileTrackerFactory.persistTrackerConfig(env.getMasterConfiguration(), builder);
+    tableDescriptor = builder.build();
+
     final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
     if (cpHost != null) {
       final RegionInfo[] regions = newRegions == null ? 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 cb30432..aa0ee27 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
@@ -610,7 +610,7 @@ public class HRegionFileSystem {
       writeRegionInfoFileContent(conf, fs, regionInfoFile, regionInfoContent);
       HRegionFileSystem regionFs = HRegionFileSystem.openRegionFromFileSystem(
         env.getMasterConfiguration(), fs, getTableDir(), regionInfo, false);
-      insertRegionFilesIntoStoreTracker(allRegionFiles, env, regionFs);
+        insertRegionFilesIntoStoreTracker(allRegionFiles, env, regionFs);
     }
     return regionDir;
   }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java
index e486e6d..483a240 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.util.Collection;
 import java.util.List;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.regionserver.StoreContext;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -85,4 +86,11 @@ class MigrationStoreFileTracker extends StoreFileTrackerBase {
     throw new UnsupportedOperationException(
       "Should not call this method on " + getClass().getSimpleName());
   }
+
+  @Override
+  public void persistConfiguration(TableDescriptorBuilder builder) {
+    super.persistConfiguration(builder);
+    builder.setValue(SRC_IMPL, src.getClass().getName());
+    builder.setValue(DST_IMPL, dst.getClass().getName());
+  }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java
index 0a85abb..81fa1a9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java
@@ -20,6 +20,8 @@ package org.apache.hadoop.hbase.regionserver.storefiletracker;
 import java.io.IOException;
 import java.util.Collection;
 import java.util.List;
+
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.regionserver.CreateStoreFileWriterParams;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
 import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
@@ -71,4 +73,10 @@ public interface StoreFileTracker {
    * @return Writer for a new StoreFile
    */
   StoreFileWriter createWriter(CreateStoreFileWriterParams params) throws IOException;
+
+  /**
+   * Saves StoreFileTracker implementations specific configs into the table descriptors.
+   * @param builder The table descriptor builder for the given table.
+   */
+  void persistConfiguration(TableDescriptorBuilder builder);
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java
index d860f8e..83ebbc7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java
@@ -17,18 +17,22 @@
  */
 package org.apache.hadoop.hbase.regionserver.storefiletracker;
 
+import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.TRACK_IMPL;
+
 import java.io.IOException;
 import java.util.Collection;
 import java.util.List;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.crypto.Encryption;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.io.hfile.HFileContext;
 import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
+import org.apache.hadoop.hbase.procedure2.util.StringUtils;
 import org.apache.hadoop.hbase.regionserver.CreateStoreFileWriterParams;
 import org.apache.hadoop.hbase.regionserver.StoreContext;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
@@ -78,6 +82,15 @@ abstract class StoreFileTrackerBase implements StoreFileTracker {
     }
   }
 
+  @Override
+  public void persistConfiguration(TableDescriptorBuilder builder) {
+    if (StringUtils.isEmpty(builder.getValue(TRACK_IMPL))) {
+      String trackerImpl = StoreFileTrackerFactory.
+        getStoreFileTrackerImpl(conf).getName();
+      builder.setValue(TRACK_IMPL, trackerImpl).build();
+    }
+  }
+
   private HFileContext createFileContext(Compression.Algorithm compression,
     boolean includeMVCCReadpoint, boolean includesTag, Encryption.Context encryptionContext) {
     if (compression == null) {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java
index 6eb269e..b9ec713 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java
@@ -6,9 +6,7 @@
  * 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
- *
+ * 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.
@@ -21,6 +19,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
 import org.apache.hadoop.hbase.regionserver.StoreContext;
 import org.apache.hadoop.hbase.regionserver.StoreUtils;
@@ -35,15 +34,17 @@ import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
 /**
  * Factory method for creating store file tracker.
  */
-@InterfaceAudience.Private
-public final class StoreFileTrackerFactory {
+@InterfaceAudience.Private public final class StoreFileTrackerFactory {
   public static final String TRACK_IMPL = "hbase.store.file-tracker.impl";
   private static final Logger LOG = LoggerFactory.getLogger(StoreFileTrackerFactory.class);
 
+  public static Class<? extends StoreFileTracker> getStoreFileTrackerImpl(Configuration conf) {
+    return conf.getClass(TRACK_IMPL, DefaultStoreFileTracker.class, StoreFileTracker.class);
+  }
+
   public static StoreFileTracker create(Configuration conf, boolean isPrimaryReplica,
     StoreContext ctx) {
-    Class<? extends StoreFileTracker> tracker =
-      conf.getClass(TRACK_IMPL, DefaultStoreFileTracker.class, StoreFileTracker.class);
+    Class<? extends StoreFileTracker> tracker = getStoreFileTrackerImpl(conf);
     LOG.info("instantiating StoreFileTracker impl {}", tracker.getName());
     return ReflectionUtils.newInstance(tracker, conf, isPrimaryReplica, ctx);
   }
@@ -54,7 +55,7 @@ public final class StoreFileTrackerFactory {
       ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(family));
     StoreContext ctx = StoreContext.getBuilder().withColumnFamilyDescriptor(fDescBuilder.build())
       .withRegionFileSystem(regionFs).build();
-    return StoreFileTrackerFactory.create(conf, TRACK_IMPL, isPrimaryReplica, ctx);
+    return StoreFileTrackerFactory.create(conf, isPrimaryReplica, ctx);
   }
 
   public static Configuration mergeConfigurations(Configuration global, TableDescriptor table,
@@ -75,4 +76,12 @@ public final class StoreFileTrackerFactory {
     LOG.info("instantiating StoreFileTracker impl {} as {}", tracker.getName(), configName);
     return ReflectionUtils.newInstance(tracker, conf, isPrimaryReplica, ctx);
   }
+
+  public static void persistTrackerConfig(Configuration conf, TableDescriptorBuilder builder) {
+    TableDescriptor tableDescriptor = builder.build();
+    ColumnFamilyDescriptor cfDesc = tableDescriptor.getColumnFamilies()[0];
+    StoreContext context = StoreContext.getBuilder().withColumnFamilyDescriptor(cfDesc).build();
+    StoreFileTracker tracker = StoreFileTrackerFactory.create(conf, true, context);
+    tracker.persistConfiguration(builder);
+  }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java
index d43dbe0..2235cb3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.TRACK_IMPL;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotEquals;
@@ -39,6 +40,7 @@ import org.apache.hadoop.hbase.TableExistsException;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.master.LoadBalancer;
+import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -420,6 +422,10 @@ public class TestAdmin extends TestAdminBase {
     assertEquals(BLOCK_SIZE, newTableDesc.getColumnFamily(FAMILY_1).getBlocksize());
     assertEquals(BLOCK_CACHE, newTableDesc.getColumnFamily(FAMILY_1).isBlockCacheEnabled());
     assertEquals(TTL, newTableDesc.getColumnFamily(FAMILY_1).getTimeToLive());
+    // HBASE-26246 introduced persist of store file tracker into table descriptor
+    tableDesc = TableDescriptorBuilder.newBuilder(tableDesc).setValue(TRACK_IMPL,
+      StoreFileTrackerFactory.getStoreFileTrackerImpl(TEST_UTIL.getConfiguration()).getName()).
+      build();
     TEST_UTIL.verifyTableDescriptorIgnoreTableName(tableDesc, newTableDesc);
 
     if (preserveSplits) {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin3.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin3.java
index e7be08e..7d40fd1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin3.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin3.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.TRACK_IMPL;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -34,6 +35,7 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotDisabledException;
 import org.apache.hadoop.hbase.TableNotEnabledException;
 import org.apache.hadoop.hbase.TableNotFoundException;
+import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -236,6 +238,10 @@ public class TestAdmin3 extends TestAdminBase {
     ADMIN.createTable(htd);
     Table table = TEST_UTIL.getConnection().getTable(htd.getTableName());
     TableDescriptor confirmedHtd = table.getDescriptor();
+    //HBASE-26246 introduced persist of store file tracker into table descriptor
+    htd = TableDescriptorBuilder.newBuilder(htd).setValue(TRACK_IMPL,
+      StoreFileTrackerFactory.getStoreFileTrackerImpl(TEST_UTIL.getConfiguration()).getName()).
+      build();
     assertEquals(0, TableDescriptor.COMPARATOR.compare(htd, confirmedHtd));
     MetaTableAccessor.fullScanMetaAndPrint(TEST_UTIL.getConnection());
     table.close();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java
index 42d0118..67b19c0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java
@@ -18,10 +18,12 @@
 package org.apache.hadoop.hbase.client;
 
 import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME;
+import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.TRACK_IMPL;
 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.ArrayList;
 import java.util.HashMap;
 import java.util.Iterator;
@@ -38,6 +40,7 @@ import org.apache.hadoop.hbase.TableExistsException;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.master.LoadBalancer;
+import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -409,6 +412,10 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
     assertEquals(BLOCK_SIZE, newTableDesc.getColumnFamily(FAMILY_1).getBlocksize());
     assertEquals(BLOCK_CACHE, newTableDesc.getColumnFamily(FAMILY_1).isBlockCacheEnabled());
     assertEquals(TTL, newTableDesc.getColumnFamily(FAMILY_1).getTimeToLive());
+    //HBASE-26246 introduced persist of store file tracker into table descriptor
+    tableDesc = TableDescriptorBuilder.newBuilder(tableDesc).setValue(TRACK_IMPL,
+      StoreFileTrackerFactory.getStoreFileTrackerImpl(TEST_UTIL.getConfiguration()).getName()).
+      build();
     TEST_UTIL.verifyTableDescriptorIgnoreTableName(tableDesc, newTableDesc);
 
     if (preserveSplits) {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi3.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi3.java
index f8b37a9..b30a08d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi3.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi3.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hbase.client;
 
 import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME;
+import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.TRACK_IMPL;
 import static org.hamcrest.CoreMatchers.instanceOf;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.junit.Assert.assertEquals;
@@ -36,6 +37,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder.ModifyableTableDescriptor;
+import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -149,7 +151,11 @@ public class TestAsyncTableAdminApi3 extends TestAsyncAdminBase {
     admin.createTable(desc).join();
     ModifyableTableDescriptor modifyableDesc = ((ModifyableTableDescriptor) desc);
     TableDescriptor confirmedHtd = admin.getDescriptor(tableName).get();
-    assertEquals(0, modifyableDesc.compareTo((ModifyableTableDescriptor) confirmedHtd));
+    //HBASE-26246 introduced persist of store file tracker into table descriptor
+    desc = TableDescriptorBuilder.newBuilder(desc).setValue(TRACK_IMPL,
+      StoreFileTrackerFactory.getStoreFileTrackerImpl(TEST_UTIL.getConfiguration()).getName()).
+      build();
+    assertEquals(0, TableDescriptor.COMPARATOR.compare(desc, confirmedHtd));
   }
 
   @Test
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 94254d9..1748f10 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
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hbase.master.procedure;
 
+import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.TRACK_IMPL;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -58,6 +59,7 @@ import org.apache.hadoop.hbase.procedure2.Procedure;
 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.regionserver.storefiletracker.StoreFileTrackerFactory;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
@@ -224,6 +226,11 @@ public class MasterProcedureTestingUtility {
       assertTrue("family not found " + family[i], htd.getColumnFamily(Bytes.toBytes(family[i])) != null);
     }
     assertEquals(family.length, htd.getColumnFamilyCount());
+
+    // checks store file tracker impl has been properly set in htd
+    String storeFileTrackerImpl =
+      StoreFileTrackerFactory.getStoreFileTrackerImpl(master.getConfiguration()).getName();
+    assertEquals(storeFileTrackerImpl, htd.getValue(TRACK_IMPL));
   }
 
   public static void validateTableDeletion(
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 3b57ed5..0bc77f0 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
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hbase.master.procedure;
 
+import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.TRACK_IMPL;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -38,6 +39,7 @@ import org.apache.hadoop.hbase.master.MasterFileSystem;
 import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
+import org.apache.hadoop.hbase.regionserver.storefiletracker.TestStoreFileTracker;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -89,6 +91,21 @@ public class TestCreateTableProcedure extends TestTableDDLProcedureBase {
   }
 
   @Test
+  public void testCreateWithTrackImpl() throws Exception {
+    final TableName tableName = TableName.valueOf(name.getMethodName());
+    ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+    TableDescriptor htd = MasterProcedureTestingUtility.createHTD(tableName, F1);
+    String trackerName = TestStoreFileTracker.class.getName();
+    htd = TableDescriptorBuilder.newBuilder(htd).setValue(TRACK_IMPL, trackerName).build();
+    RegionInfo[] regions = ModifyRegionUtils.createRegionInfos(htd, null);
+    long procId = ProcedureTestingUtility.submitAndWait(procExec,
+      new CreateTableProcedure(procExec.getEnvironment(), htd, regions));
+    ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId));
+    htd = getMaster().getTableDescriptors().get(tableName);
+    assertEquals(trackerName, htd.getValue(TRACK_IMPL));
+  }
+
+  @Test
   public void testCreateWithoutColumnFamily() throws Exception {
     final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
     final TableName tableName = TableName.valueOf(name.getMethodName());
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestMigrationStoreFileTracker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestMigrationStoreFileTracker.java
index e9ebb4c..567adf0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestMigrationStoreFileTracker.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestMigrationStoreFileTracker.java
@@ -29,7 +29,7 @@ import java.util.stream.Collectors;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
-import org.apache.hadoop.hbase.HBaseTestingUtil;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Get;
@@ -70,7 +70,7 @@ public class TestMigrationStoreFileTracker {
   public static final HBaseClassTestRule CLASS_RULE =
     HBaseClassTestRule.forClass(TestMigrationStoreFileTracker.class);
 
-  private static final HBaseTestingUtil UTIL = new HBaseTestingUtil();
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
 
   private static final byte[] CF = Bytes.toBytes("cf");
 
@@ -125,7 +125,7 @@ public class TestMigrationStoreFileTracker {
     conf.setClass(MigrationStoreFileTracker.SRC_IMPL, srcImplClass, StoreFileTrackerBase.class);
     conf.setClass(MigrationStoreFileTracker.DST_IMPL, dstImplClass, StoreFileTrackerBase.class);
     rootDir = UTIL.getDataTestDir(name.getMethodName().replaceAll("[=:\\[ ]", "_"));
-    wal = HBaseTestingUtil.createWal(conf, rootDir, RI);
+    wal = HBaseTestingUtility.createWal(conf, rootDir, RI);
   }
 
   @After
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTracker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTracker.java
index 05ca1fc..1dc9c4e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTracker.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTracker.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.regionserver.storefiletracker;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -27,6 +28,7 @@ import java.util.Map;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.regionserver.StoreContext;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
+import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -38,15 +40,21 @@ public class TestStoreFileTracker extends DefaultStoreFileTracker {
 
   public TestStoreFileTracker(Configuration conf, boolean isPrimaryReplica, StoreContext ctx) {
     super(conf, isPrimaryReplica, ctx);
-    this.storeId = ctx.getRegionInfo().getEncodedName() + "-" + ctx.getFamily().getNameAsString();
-    LOG.info("created storeId: {}", storeId);
-    trackedFiles.computeIfAbsent(storeId, v -> new ArrayList<>());
+    if (ctx.getRegionFileSystem() != null) {
+      this.storeId = ctx.getRegionInfo().getEncodedName() + "-" + ctx.getFamily().getNameAsString();
+      LOG.info("created storeId: {}", storeId);
+      trackedFiles.computeIfAbsent(storeId, v -> new ArrayList<>());
+    } else {
+      LOG.info("ctx.getRegionFileSystem() returned null. Leaving storeId null.");
+    }
+
   }
 
   @Override
   protected void doAddNewStoreFiles(Collection<StoreFileInfo> newFiles) throws IOException {
     LOG.info("adding to storeId: {}", storeId);
     trackedFiles.get(storeId).addAll(newFiles);
+    trackedFiles.putIfAbsent(storeId, (List<StoreFileInfo>)newFiles);
   }
 
   @Override

[hbase] 06/15: HBASE-26248 Should find a suitable way to let users specify the store file tracker implementation (#3665)

Posted by zh...@apache.org.
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

commit 49c40998ef8e1a8d8357a2f31876d70bb325c491
Author: Duo Zhang <zh...@apache.org>
AuthorDate: Tue Sep 14 16:28:21 2021 +0800

    HBASE-26248 Should find a suitable way to let users specify the store file tracker implementation (#3665)
    
    Signed-off-by: Wellington Chevreuil <wc...@apache.org>
---
 .../MigrationStoreFileTracker.java                 | 13 ++-
 .../storefiletracker/StoreFileTracker.java         |  7 +-
 .../storefiletracker/StoreFileTrackerBase.java     | 12 +--
 .../storefiletracker/StoreFileTrackerFactory.java  | 97 +++++++++++++++++++---
 .../org/apache/hadoop/hbase/client/TestAdmin.java  |  6 +-
 .../org/apache/hadoop/hbase/client/TestAdmin3.java |  6 +-
 .../hbase/client/TestAsyncTableAdminApi.java       |  6 +-
 .../hbase/client/TestAsyncTableAdminApi3.java      |  6 +-
 .../procedure/MasterProcedureTestingUtility.java   |  6 +-
 .../master/procedure/TestCreateTableProcedure.java |  6 +-
 .../regionserver/TestMergesSplitsAddToTracker.java |  4 +-
 .../TestMigrationStoreFileTracker.java             | 27 +++---
 .../TestRegionWithFileBasedStoreFileTracker.java   |  3 +-
 .../TestStoreFileTrackerFactory.java               | 58 +++++++++++++
 14 files changed, 202 insertions(+), 55 deletions(-)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java
index 483a240..3eeef90 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java
@@ -22,6 +22,7 @@ import java.util.Collection;
 import java.util.List;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.procedure2.util.StringUtils;
 import org.apache.hadoop.hbase.regionserver.StoreContext;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -44,8 +45,8 @@ class MigrationStoreFileTracker extends StoreFileTrackerBase {
 
   public MigrationStoreFileTracker(Configuration conf, boolean isPrimaryReplica, StoreContext ctx) {
     super(conf, isPrimaryReplica, ctx);
-    this.src = StoreFileTrackerFactory.create(conf, SRC_IMPL, isPrimaryReplica, ctx);
-    this.dst = StoreFileTrackerFactory.create(conf, DST_IMPL, isPrimaryReplica, ctx);
+    this.src = StoreFileTrackerFactory.createForMigration(conf, SRC_IMPL, isPrimaryReplica, ctx);
+    this.dst = StoreFileTrackerFactory.createForMigration(conf, DST_IMPL, isPrimaryReplica, ctx);
     Preconditions.checkArgument(!src.getClass().equals(dst.getClass()),
       "src and dst is the same: %s", src.getClass());
   }
@@ -90,7 +91,11 @@ class MigrationStoreFileTracker extends StoreFileTrackerBase {
   @Override
   public void persistConfiguration(TableDescriptorBuilder builder) {
     super.persistConfiguration(builder);
-    builder.setValue(SRC_IMPL, src.getClass().getName());
-    builder.setValue(DST_IMPL, dst.getClass().getName());
+    if (StringUtils.isEmpty(builder.getValue(SRC_IMPL))) {
+      builder.setValue(SRC_IMPL, src.getTrackerName());
+    }
+    if (StringUtils.isEmpty(builder.getValue(DST_IMPL))) {
+      builder.setValue(DST_IMPL, dst.getTrackerName());
+    }
   }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java
index 81fa1a9..59fe7ef 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java
@@ -75,7 +75,12 @@ public interface StoreFileTracker {
   StoreFileWriter createWriter(CreateStoreFileWriterParams params) throws IOException;
 
   /**
-   * Saves StoreFileTracker implementations specific configs into the table descriptors.
+   * Saves StoreFileTracker implementations specific configurations into the table descriptors.
+   * <p/>
+   * This is used to avoid accidentally data loss when changing the cluster level store file tracker
+   * implementation, and also possible misconfiguration between master and region servers.
+   * <p/>
+   * See HBASE-26246 for more details.
    * @param builder The table descriptor builder for the given table.
    */
   void persistConfiguration(TableDescriptorBuilder builder);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java
index 83ebbc7..a786add 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java
@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hbase.regionserver.storefiletracker;
 
-import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.TRACK_IMPL;
+import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.TRACKER_IMPL;
 
 import java.io.IOException;
 import java.util.Collection;
@@ -84,13 +84,15 @@ abstract class StoreFileTrackerBase implements StoreFileTracker {
 
   @Override
   public void persistConfiguration(TableDescriptorBuilder builder) {
-    if (StringUtils.isEmpty(builder.getValue(TRACK_IMPL))) {
-      String trackerImpl = StoreFileTrackerFactory.
-        getStoreFileTrackerImpl(conf).getName();
-      builder.setValue(TRACK_IMPL, trackerImpl).build();
+    if (StringUtils.isEmpty(builder.getValue(TRACKER_IMPL))) {
+      builder.setValue(TRACKER_IMPL, getTrackerName());
     }
   }
 
+  protected final String getTrackerName() {
+    return StoreFileTrackerFactory.getStoreFileTrackerName(getClass());
+  }
+
   private HFileContext createFileContext(Compression.Algorithm compression,
     boolean includeMVCCReadpoint, boolean includesTag, Encryption.Context encryptionContext) {
     if (compression == null) {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java
index b9ec713..9be19ec 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java
@@ -15,6 +15,9 @@
  */
 package org.apache.hadoop.hbase.regionserver.storefiletracker;
 
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
@@ -33,22 +36,81 @@ import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
 
 /**
  * Factory method for creating store file tracker.
+ * <p/>
+ * The current implementations are:
+ * <ul>
+ * <li><em>default</em>: DefaultStoreFileTracker, see {@link DefaultStoreFileTracker}.</li>
+ * <li><em>file</em>:FileBasedStoreFileTracker, see {@link FileBasedStoreFileTracker}.</li>
+ * <li><em>migration</em>:MigrationStoreFileTracker, see {@link MigrationStoreFileTracker}.</li>
+ * </ul>
+ * @see DefaultStoreFileTracker
+ * @see FileBasedStoreFileTracker
+ * @see MigrationStoreFileTracker
  */
-@InterfaceAudience.Private public final class StoreFileTrackerFactory {
-  public static final String TRACK_IMPL = "hbase.store.file-tracker.impl";
+@InterfaceAudience.Private
+public final class StoreFileTrackerFactory {
+
   private static final Logger LOG = LoggerFactory.getLogger(StoreFileTrackerFactory.class);
 
-  public static Class<? extends StoreFileTracker> getStoreFileTrackerImpl(Configuration conf) {
-    return conf.getClass(TRACK_IMPL, DefaultStoreFileTracker.class, StoreFileTracker.class);
+  public static final String TRACKER_IMPL = "hbase.store.file-tracker.impl";
+
+  /**
+   * Maps between configuration names for trackers and implementation classes.
+   */
+  public enum Trackers {
+    DEFAULT(DefaultStoreFileTracker.class), FILE(FileBasedStoreFileTracker.class),
+    MIGRATION(MigrationStoreFileTracker.class);
+
+    final Class<? extends StoreFileTracker> clazz;
+
+    Trackers(Class<? extends StoreFileTracker> clazz) {
+      this.clazz = clazz;
+    }
+  }
+
+  private static final Map<Class<? extends StoreFileTracker>, Trackers> CLASS_TO_ENUM = reverse();
+
+  private static Map<Class<? extends StoreFileTracker>, Trackers> reverse() {
+    Map<Class<? extends StoreFileTracker>, Trackers> map = new HashMap<>();
+    for (Trackers tracker : Trackers.values()) {
+      map.put(tracker.clazz, tracker);
+    }
+    return Collections.unmodifiableMap(map);
+  }
+
+  private StoreFileTrackerFactory() {
+  }
+
+  public static String getStoreFileTrackerName(Configuration conf) {
+    return conf.get(TRACKER_IMPL, Trackers.DEFAULT.name());
+  }
+
+  static String getStoreFileTrackerName(Class<? extends StoreFileTracker> clazz) {
+    Trackers name = CLASS_TO_ENUM.get(clazz);
+    return name != null ? name.name() : clazz.getName();
+  }
+
+  private static Class<? extends StoreFileTracker> getTrackerClass(Configuration conf) {
+    try {
+      Trackers tracker = Trackers.valueOf(getStoreFileTrackerName(conf).toUpperCase());
+      return tracker.clazz;
+    } catch (IllegalArgumentException e) {
+      // Fall back to them specifying a class name
+      return conf.getClass(TRACKER_IMPL, Trackers.DEFAULT.clazz, StoreFileTracker.class);
+    }
   }
 
   public static StoreFileTracker create(Configuration conf, boolean isPrimaryReplica,
     StoreContext ctx) {
-    Class<? extends StoreFileTracker> tracker = getStoreFileTrackerImpl(conf);
+    Class<? extends StoreFileTracker> tracker = getTrackerClass(conf);
     LOG.info("instantiating StoreFileTracker impl {}", tracker.getName());
     return ReflectionUtils.newInstance(tracker, conf, isPrimaryReplica, ctx);
   }
 
+  /**
+   * Used at master side when splitting/merging regions, as we do not have a Store, thus no
+   * StoreContext at master side.
+   */
   public static StoreFileTracker create(Configuration conf, boolean isPrimaryReplica, String family,
     HRegionFileSystem regionFs) {
     ColumnFamilyDescriptorBuilder fDescBuilder =
@@ -63,15 +125,30 @@ import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
     return StoreUtils.createStoreConfiguration(global, table, family);
   }
 
-  static StoreFileTrackerBase create(Configuration conf, String configName,
+  /**
+   * Create store file tracker to be used as source or destination for
+   * {@link MigrationStoreFileTracker}.
+   */
+  static StoreFileTrackerBase createForMigration(Configuration conf, String configName,
     boolean isPrimaryReplica, StoreContext ctx) {
-    String className =
+    String trackerName =
       Preconditions.checkNotNull(conf.get(configName), "config %s is not set", configName);
     Class<? extends StoreFileTrackerBase> tracker;
     try {
-      tracker = Class.forName(className).asSubclass(StoreFileTrackerBase.class);
-    } catch (ClassNotFoundException e) {
-      throw new RuntimeException(e);
+      tracker =
+        Trackers.valueOf(trackerName.toUpperCase()).clazz.asSubclass(StoreFileTrackerBase.class);
+    } catch (IllegalArgumentException e) {
+      // Fall back to them specifying a class name
+      try {
+        tracker = Class.forName(trackerName).asSubclass(StoreFileTrackerBase.class);
+      } catch (ClassNotFoundException cnfe) {
+        throw new RuntimeException(cnfe);
+      }
+    }
+    // prevent nest of MigrationStoreFileTracker, it will cause infinite recursion.
+    if (MigrationStoreFileTracker.class.isAssignableFrom(tracker)) {
+      throw new IllegalArgumentException("Should not specify " + configName + " as " +
+        Trackers.MIGRATION + " because it can not be nested");
     }
     LOG.info("instantiating StoreFileTracker impl {} as {}", tracker.getName(), configName);
     return ReflectionUtils.newInstance(tracker, conf, isPrimaryReplica, ctx);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java
index 2235cb3..a8e4b7e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java
@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hbase.client;
 
-import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.TRACK_IMPL;
+import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.TRACKER_IMPL;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotEquals;
@@ -423,8 +423,8 @@ public class TestAdmin extends TestAdminBase {
     assertEquals(BLOCK_CACHE, newTableDesc.getColumnFamily(FAMILY_1).isBlockCacheEnabled());
     assertEquals(TTL, newTableDesc.getColumnFamily(FAMILY_1).getTimeToLive());
     // HBASE-26246 introduced persist of store file tracker into table descriptor
-    tableDesc = TableDescriptorBuilder.newBuilder(tableDesc).setValue(TRACK_IMPL,
-      StoreFileTrackerFactory.getStoreFileTrackerImpl(TEST_UTIL.getConfiguration()).getName()).
+    tableDesc = TableDescriptorBuilder.newBuilder(tableDesc).setValue(TRACKER_IMPL,
+      StoreFileTrackerFactory.getStoreFileTrackerName(TEST_UTIL.getConfiguration())).
       build();
     TEST_UTIL.verifyTableDescriptorIgnoreTableName(tableDesc, newTableDesc);
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin3.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin3.java
index 7d40fd1..c2de0fb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin3.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin3.java
@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hbase.client;
 
-import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.TRACK_IMPL;
+import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.TRACKER_IMPL;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -239,8 +239,8 @@ public class TestAdmin3 extends TestAdminBase {
     Table table = TEST_UTIL.getConnection().getTable(htd.getTableName());
     TableDescriptor confirmedHtd = table.getDescriptor();
     //HBASE-26246 introduced persist of store file tracker into table descriptor
-    htd = TableDescriptorBuilder.newBuilder(htd).setValue(TRACK_IMPL,
-      StoreFileTrackerFactory.getStoreFileTrackerImpl(TEST_UTIL.getConfiguration()).getName()).
+    htd = TableDescriptorBuilder.newBuilder(htd).setValue(TRACKER_IMPL,
+      StoreFileTrackerFactory.getStoreFileTrackerName(TEST_UTIL.getConfiguration())).
       build();
     assertEquals(0, TableDescriptor.COMPARATOR.compare(htd, confirmedHtd));
     MetaTableAccessor.fullScanMetaAndPrint(TEST_UTIL.getConnection());
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java
index 67b19c0..8792f89 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java
@@ -18,7 +18,7 @@
 package org.apache.hadoop.hbase.client;
 
 import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME;
-import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.TRACK_IMPL;
+import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.TRACKER_IMPL;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -413,8 +413,8 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
     assertEquals(BLOCK_CACHE, newTableDesc.getColumnFamily(FAMILY_1).isBlockCacheEnabled());
     assertEquals(TTL, newTableDesc.getColumnFamily(FAMILY_1).getTimeToLive());
     //HBASE-26246 introduced persist of store file tracker into table descriptor
-    tableDesc = TableDescriptorBuilder.newBuilder(tableDesc).setValue(TRACK_IMPL,
-      StoreFileTrackerFactory.getStoreFileTrackerImpl(TEST_UTIL.getConfiguration()).getName()).
+    tableDesc = TableDescriptorBuilder.newBuilder(tableDesc).setValue(TRACKER_IMPL,
+      StoreFileTrackerFactory.getStoreFileTrackerName(TEST_UTIL.getConfiguration())).
       build();
     TEST_UTIL.verifyTableDescriptorIgnoreTableName(tableDesc, newTableDesc);
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi3.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi3.java
index b30a08d..d2550fc 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi3.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi3.java
@@ -18,7 +18,7 @@
 package org.apache.hadoop.hbase.client;
 
 import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME;
-import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.TRACK_IMPL;
+import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.TRACKER_IMPL;
 import static org.hamcrest.CoreMatchers.instanceOf;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.junit.Assert.assertEquals;
@@ -152,8 +152,8 @@ public class TestAsyncTableAdminApi3 extends TestAsyncAdminBase {
     ModifyableTableDescriptor modifyableDesc = ((ModifyableTableDescriptor) desc);
     TableDescriptor confirmedHtd = admin.getDescriptor(tableName).get();
     //HBASE-26246 introduced persist of store file tracker into table descriptor
-    desc = TableDescriptorBuilder.newBuilder(desc).setValue(TRACK_IMPL,
-      StoreFileTrackerFactory.getStoreFileTrackerImpl(TEST_UTIL.getConfiguration()).getName()).
+    desc = TableDescriptorBuilder.newBuilder(desc).setValue(TRACKER_IMPL,
+      StoreFileTrackerFactory.getStoreFileTrackerName(TEST_UTIL.getConfiguration())).
       build();
     assertEquals(0, TableDescriptor.COMPARATOR.compare(desc, confirmedHtd));
   }
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 1748f10..210b275 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
@@ -18,7 +18,7 @@
 
 package org.apache.hadoop.hbase.master.procedure;
 
-import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.TRACK_IMPL;
+import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.TRACKER_IMPL;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -229,8 +229,8 @@ public class MasterProcedureTestingUtility {
 
     // checks store file tracker impl has been properly set in htd
     String storeFileTrackerImpl =
-      StoreFileTrackerFactory.getStoreFileTrackerImpl(master.getConfiguration()).getName();
-    assertEquals(storeFileTrackerImpl, htd.getValue(TRACK_IMPL));
+      StoreFileTrackerFactory.getStoreFileTrackerName(master.getConfiguration());
+    assertEquals(storeFileTrackerImpl, htd.getValue(TRACKER_IMPL));
   }
 
   public static void validateTableDeletion(
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 0bc77f0..f432c80 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
@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hbase.master.procedure;
 
-import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.TRACK_IMPL;
+import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.TRACKER_IMPL;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -96,13 +96,13 @@ public class TestCreateTableProcedure extends TestTableDDLProcedureBase {
     ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
     TableDescriptor htd = MasterProcedureTestingUtility.createHTD(tableName, F1);
     String trackerName = TestStoreFileTracker.class.getName();
-    htd = TableDescriptorBuilder.newBuilder(htd).setValue(TRACK_IMPL, trackerName).build();
+    htd = TableDescriptorBuilder.newBuilder(htd).setValue(TRACKER_IMPL, trackerName).build();
     RegionInfo[] regions = ModifyRegionUtils.createRegionInfos(htd, null);
     long procId = ProcedureTestingUtility.submitAndWait(procExec,
       new CreateTableProcedure(procExec.getEnvironment(), htd, regions));
     ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId));
     htd = getMaster().getTableDescriptors().get(tableName);
-    assertEquals(trackerName, htd.getValue(TRACK_IMPL));
+    assertEquals(trackerName, htd.getValue(TRACKER_IMPL));
   }
 
   @Test
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMergesSplitsAddToTracker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMergesSplitsAddToTracker.java
index 6a9e08f..435fa26 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMergesSplitsAddToTracker.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMergesSplitsAddToTracker.java
@@ -18,7 +18,7 @@
 package org.apache.hadoop.hbase.regionserver;
 
 import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.
-  TRACK_IMPL;
+  TRACKER_IMPL;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
@@ -73,7 +73,7 @@ public class TestMergesSplitsAddToTracker {
 
   @BeforeClass
   public static void setupClass() throws Exception {
-    TEST_UTIL.getConfiguration().set(TRACK_IMPL, TestStoreFileTracker.class.getName());
+    TEST_UTIL.getConfiguration().set(TRACKER_IMPL, TestStoreFileTracker.class.getName());
     TEST_UTIL.startMiniCluster();
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestMigrationStoreFileTracker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestMigrationStoreFileTracker.java
index 567adf0..c6f51ff 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestMigrationStoreFileTracker.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestMigrationStoreFileTracker.java
@@ -23,7 +23,6 @@ import static org.junit.Assert.assertEquals;
 
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.List;
 import java.util.stream.Collectors;
 import org.apache.hadoop.conf.Configuration;
@@ -86,10 +85,10 @@ public class TestMigrationStoreFileTracker {
   public TestName name = new TestName();
 
   @Parameter(0)
-  public Class<? extends StoreFileTrackerBase> srcImplClass;
+  public StoreFileTrackerFactory.Trackers srcImpl;
 
   @Parameter(1)
-  public Class<? extends StoreFileTrackerBase> dstImplClass;
+  public StoreFileTrackerFactory.Trackers dstImpl;
 
   private HRegion region;
 
@@ -99,11 +98,13 @@ public class TestMigrationStoreFileTracker {
 
   @Parameters(name = "{index}: src={0}, dst={1}")
   public static List<Object[]> params() {
-    List<Class<? extends StoreFileTrackerBase>> impls =
-      Arrays.asList(DefaultStoreFileTracker.class, FileBasedStoreFileTracker.class);
     List<Object[]> params = new ArrayList<>();
-    for (Class<? extends StoreFileTrackerBase> src : impls) {
-      for (Class<? extends StoreFileTrackerBase> dst : impls) {
+    for (StoreFileTrackerFactory.Trackers src : StoreFileTrackerFactory.Trackers.values()) {
+      for (StoreFileTrackerFactory.Trackers dst : StoreFileTrackerFactory.Trackers.values()) {
+        if (src == StoreFileTrackerFactory.Trackers.MIGRATION
+          || dst == StoreFileTrackerFactory.Trackers.MIGRATION) {
+          continue;
+        }
         if (src.equals(dst)) {
           continue;
         }
@@ -122,8 +123,8 @@ public class TestMigrationStoreFileTracker {
   @Before
   public void setUp() throws IOException {
     Configuration conf = UTIL.getConfiguration();
-    conf.setClass(MigrationStoreFileTracker.SRC_IMPL, srcImplClass, StoreFileTrackerBase.class);
-    conf.setClass(MigrationStoreFileTracker.DST_IMPL, dstImplClass, StoreFileTrackerBase.class);
+    conf.set(MigrationStoreFileTracker.SRC_IMPL, srcImpl.name().toLowerCase());
+    conf.set(MigrationStoreFileTracker.DST_IMPL, dstImpl.name().toLowerCase());
     rootDir = UTIL.getDataTestDir(name.getMethodName().replaceAll("[=:\\[ ]", "_"));
     wal = HBaseTestingUtility.createWal(conf, rootDir, RI);
   }
@@ -145,7 +146,7 @@ public class TestMigrationStoreFileTracker {
   private HRegion createRegion(Class<? extends StoreFileTrackerBase> trackerImplClass)
     throws IOException {
     Configuration conf = new Configuration(UTIL.getConfiguration());
-    conf.setClass(StoreFileTrackerFactory.TRACK_IMPL, trackerImplClass, StoreFileTracker.class);
+    conf.setClass(StoreFileTrackerFactory.TRACKER_IMPL, trackerImplClass, StoreFileTracker.class);
     return HRegion.createHRegion(RI, rootDir, conf, TD, wal, true);
   }
 
@@ -155,7 +156,7 @@ public class TestMigrationStoreFileTracker {
     List<String> before = getStoreFiles();
     region.close();
     Configuration conf = new Configuration(UTIL.getConfiguration());
-    conf.setClass(StoreFileTrackerFactory.TRACK_IMPL, trackerImplClass, StoreFileTracker.class);
+    conf.setClass(StoreFileTrackerFactory.TRACKER_IMPL, trackerImplClass, StoreFileTracker.class);
     region = HRegion.openHRegion(rootDir, RI, TD, wal, conf);
     List<String> after = getStoreFiles();
     assertEquals(before.size(), after.size());
@@ -180,14 +181,14 @@ public class TestMigrationStoreFileTracker {
 
   @Test
   public void testMigration() throws IOException {
-    region = createRegion(srcImplClass);
+    region = createRegion(srcImpl.clazz.asSubclass(StoreFileTrackerBase.class));
     putData(0, 100);
     verifyData(0, 100);
     reopenRegion(MigrationStoreFileTracker.class);
     verifyData(0, 100);
     region.compact(true);
     putData(100, 200);
-    reopenRegion(dstImplClass);
+    reopenRegion(dstImpl.clazz.asSubclass(StoreFileTrackerBase.class));
     verifyData(0, 200);
   }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestRegionWithFileBasedStoreFileTracker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestRegionWithFileBasedStoreFileTracker.java
index a434934..ee86d70 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestRegionWithFileBasedStoreFileTracker.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestRegionWithFileBasedStoreFileTracker.java
@@ -71,8 +71,7 @@ public class TestRegionWithFileBasedStoreFileTracker {
   @Before
   public void setUp() throws IOException {
     Configuration conf = new Configuration(UTIL.getConfiguration());
-    conf.setClass(StoreFileTrackerFactory.TRACK_IMPL, FileBasedStoreFileTracker.class,
-      StoreFileTracker.class);
+    conf.set(StoreFileTrackerFactory.TRACKER_IMPL, StoreFileTrackerFactory.Trackers.FILE.name());
     region = HBaseTestingUtility.createRegionAndWAL(RI, UTIL.getDataTestDir(name.getMethodName()),
       conf, TD);
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTrackerFactory.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTrackerFactory.java
new file mode 100644
index 0000000..41f2afd
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTrackerFactory.java
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.regionserver.storefiletracker;
+
+import static org.junit.Assert.assertThrows;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.regionserver.StoreContext;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ RegionServerTests.class, SmallTests.class })
+public class TestStoreFileTrackerFactory {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestStoreFileTrackerFactory.class);
+
+  @Test
+  public void testCreateForMigration() {
+    Configuration conf = HBaseConfiguration.create();
+    String configName = "config";
+
+    // no config
+    assertThrows(NullPointerException.class, () -> StoreFileTrackerFactory.createForMigration(conf,
+      configName, false, StoreContext.getBuilder().build()));
+
+    // class not found
+    conf.set(configName, "config");
+    assertThrows(RuntimeException.class, () -> StoreFileTrackerFactory.createForMigration(conf,
+      configName, false, StoreContext.getBuilder().build()));
+
+    // nested MigrationStoreFileTracker
+    conf.setClass(configName, MigrationStoreFileTracker.class, StoreFileTrackerBase.class);
+    assertThrows(IllegalArgumentException.class, () -> StoreFileTrackerFactory
+      .createForMigration(conf, configName, false, StoreContext.getBuilder().build()));
+  }
+}

[hbase] 01/15: HBASE-26064 Introduce a StoreFileTracker to abstract the store file tracking logic

Posted by zh...@apache.org.
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

commit 04e1980dea14968ff7f5357659099358204d37cf
Author: Duo Zhang <zh...@apache.org>
AuthorDate: Thu Jul 29 18:35:19 2021 +0800

    HBASE-26064 Introduce a StoreFileTracker to abstract the store file tracking logic
    
    Signed-off-by: Wellington Chevreuil <wc...@apache.org>
---
 .../hadoop/hbase/mob/DefaultMobStoreCompactor.java |  24 +-
 .../hadoop/hbase/mob/DefaultMobStoreFlusher.java   |   4 +-
 .../regionserver/CreateStoreFileWriterParams.java  | 134 ++++
 .../hbase/regionserver/DateTieredStoreEngine.java  |   5 +-
 .../hbase/regionserver/DefaultStoreEngine.java     |   5 +-
 .../hbase/regionserver/DefaultStoreFlusher.java    |  11 +-
 .../hadoop/hbase/regionserver/HMobStore.java       |   3 +-
 .../hbase/regionserver/HRegionFileSystem.java      |  10 +-
 .../apache/hadoop/hbase/regionserver/HStore.java   | 729 +++++----------------
 .../hadoop/hbase/regionserver/StoreContext.java    |   9 +
 .../hadoop/hbase/regionserver/StoreEngine.java     | 461 ++++++++++++-
 .../hbase/regionserver/StoreFileManager.java       |   9 +
 .../hadoop/hbase/regionserver/StoreFlusher.java    |   9 +-
 .../hadoop/hbase/regionserver/StoreUtils.java      |  37 +-
 .../hbase/regionserver/StripeStoreEngine.java      |   9 +-
 .../hbase/regionserver/StripeStoreFlusher.java     |   9 +-
 .../compactions/AbstractMultiOutputCompactor.java  |   7 +-
 .../hbase/regionserver/compactions/Compactor.java  |  36 +-
 .../regionserver/compactions/DefaultCompactor.java |  16 +-
 .../storefiletracker/DefaultStoreFileTracker.java  |  61 ++
 .../storefiletracker/StoreFileTracker.java         |  75 +++
 .../storefiletracker/StoreFileTrackerBase.java     | 178 +++++
 .../storefiletracker/StoreFileTrackerFactory.java  |  35 +
 .../util/compaction/MajorCompactionRequest.java    |   1 -
 .../org/apache/hadoop/hbase/TestIOFencing.java     |  12 +-
 .../regionserver/TestCacheOnWriteInSchema.java     |   6 +-
 .../hbase/regionserver/TestDefaultStoreEngine.java |   9 +-
 .../hadoop/hbase/regionserver/TestHRegion.java     |   4 +-
 .../hadoop/hbase/regionserver/TestHStore.java      |  33 +-
 .../TestRegionMergeTransactionOnCluster.java       |   6 +-
 .../regionserver/TestStoreFileRefresherChore.java  |   3 +-
 .../regionserver/TestStoreScannerClosure.java      |   6 +-
 .../hbase/regionserver/TestStripeStoreEngine.java  |  12 +-
 .../compactions/TestDateTieredCompactor.java       |  16 +-
 .../compactions/TestStripeCompactionPolicy.java    |  12 +-
 .../compactions/TestStripeCompactor.java           |  16 +-
 36 files changed, 1273 insertions(+), 739 deletions(-)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java
index 75e2811..1b218b4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java
@@ -22,13 +22,12 @@ import java.io.InterruptedIOException;
 import java.util.ArrayList;
 import java.util.Date;
 import java.util.List;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.PrivateCellUtil;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.PrivateCellUtil;
 import org.apache.hadoop.hbase.regionserver.CellSink;
 import org.apache.hadoop.hbase.regionserver.HMobStore;
 import org.apache.hadoop.hbase.regionserver.HStore;
@@ -80,17 +79,16 @@ public class DefaultMobStoreCompactor extends DefaultCompactor {
   };
 
   private final CellSinkFactory<StoreFileWriter> writerFactory =
-      new CellSinkFactory<StoreFileWriter>() {
-        @Override
-        public StoreFileWriter createWriter(InternalScanner scanner,
-            org.apache.hadoop.hbase.regionserver.compactions.Compactor.FileDetails fd,
-            boolean shouldDropBehind, boolean major) throws IOException {
-          // make this writer with tags always because of possible new cells with tags.
-          return store.createWriterInTmp(fd.maxKeyCount, 
-            major ? majorCompactionCompression : minorCompactionCompression, true, true, true,
-            shouldDropBehind);
-        }
-      };
+    new CellSinkFactory<StoreFileWriter>() {
+      @Override
+      public StoreFileWriter createWriter(InternalScanner scanner,
+        org.apache.hadoop.hbase.regionserver.compactions.Compactor.FileDetails fd,
+        boolean shouldDropBehind, boolean major) throws IOException {
+        // make this writer with tags always because of possible new cells with tags.
+        return store.getStoreEngine().createWriter(
+          createParams(fd, shouldDropBehind, major).includeMVCCReadpoint(true).includesTag(true));
+      }
+    };
 
   public DefaultMobStoreCompactor(Configuration conf, HStore store) {
     super(conf, store);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreFlusher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreFlusher.java
index cc610e5..a52ce2b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreFlusher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreFlusher.java
@@ -23,7 +23,6 @@ import java.io.InterruptedIOException;
 import java.util.ArrayList;
 import java.util.Date;
 import java.util.List;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
@@ -115,8 +114,7 @@ public class DefaultMobStoreFlusher extends DefaultStoreFlusher {
       synchronized (flushLock) {
         status.setStatus("Flushing " + store + ": creating writer");
         // Write the map out to the disk
-        writer = store.createWriterInTmp(cellsCount, store.getColumnFamilyDescriptor().getCompressionType(),
-            false, true, true, false);
+        writer = createWriter(snapshot, true);
         IOException e = null;
         try {
           // It's a mob store, flush the cells in a mob way. This is the difference of flushing
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CreateStoreFileWriterParams.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CreateStoreFileWriterParams.java
new file mode 100644
index 0000000..10cd9f0
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CreateStoreFileWriterParams.java
@@ -0,0 +1,134 @@
+/**
+ * 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.regionserver;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.io.compress.Compression;
+import org.apache.yetus.audience.InterfaceAudience;
+
+@InterfaceAudience.Private
+public final class CreateStoreFileWriterParams {
+
+  private long maxKeyCount;
+
+  private Compression.Algorithm compression;
+
+  private boolean isCompaction;
+
+  private boolean includeMVCCReadpoint;
+
+  private boolean includesTag;
+
+  private boolean shouldDropBehind;
+
+  private long totalCompactedFilesSize = -1;
+
+  private String fileStoragePolicy = HConstants.EMPTY_STRING;
+
+  private CreateStoreFileWriterParams() {
+  }
+
+  public long maxKeyCount() {
+    return maxKeyCount;
+  }
+
+  public CreateStoreFileWriterParams maxKeyCount(long maxKeyCount) {
+    this.maxKeyCount = maxKeyCount;
+    return this;
+  }
+
+  public Compression.Algorithm compression() {
+    return compression;
+  }
+
+  /**
+   * Set the compression algorithm to use
+   */
+  public CreateStoreFileWriterParams compression(Compression.Algorithm compression) {
+    this.compression = compression;
+    return this;
+  }
+
+  public boolean isCompaction() {
+    return isCompaction;
+  }
+
+  /**
+   * Whether we are creating a new file in a compaction
+   */
+  public CreateStoreFileWriterParams isCompaction(boolean isCompaction) {
+    this.isCompaction = isCompaction;
+    return this;
+  }
+
+  public boolean includeMVCCReadpoint() {
+    return includeMVCCReadpoint;
+  }
+
+  /**
+   * Whether to include MVCC or not
+   */
+  public CreateStoreFileWriterParams includeMVCCReadpoint(boolean includeMVCCReadpoint) {
+    this.includeMVCCReadpoint = includeMVCCReadpoint;
+    return this;
+  }
+
+  public boolean includesTag() {
+    return includesTag;
+  }
+
+  /**
+   * Whether to includesTag or not
+   */
+  public CreateStoreFileWriterParams includesTag(boolean includesTag) {
+    this.includesTag = includesTag;
+    return this;
+  }
+
+  public boolean shouldDropBehind() {
+    return shouldDropBehind;
+  }
+
+  public CreateStoreFileWriterParams shouldDropBehind(boolean shouldDropBehind) {
+    this.shouldDropBehind = shouldDropBehind;
+    return this;
+  }
+
+  public long totalCompactedFilesSize() {
+    return totalCompactedFilesSize;
+  }
+
+  public CreateStoreFileWriterParams totalCompactedFilesSize(long totalCompactedFilesSize) {
+    this.totalCompactedFilesSize = totalCompactedFilesSize;
+    return this;
+  }
+
+  public String fileStoragePolicy() {
+    return fileStoragePolicy;
+  }
+
+  public CreateStoreFileWriterParams fileStoragePolicy(String fileStoragePolicy) {
+    this.fileStoragePolicy = fileStoragePolicy;
+    return this;
+  }
+
+  public static CreateStoreFileWriterParams create() {
+    return new CreateStoreFileWriterParams();
+  }
+
+}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DateTieredStoreEngine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DateTieredStoreEngine.java
index 1df953d..7422d91 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DateTieredStoreEngine.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DateTieredStoreEngine.java
@@ -19,18 +19,17 @@ package org.apache.hadoop.hbase.regionserver;
 
 import java.io.IOException;
 import java.util.List;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.CellComparator;
-import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequestImpl;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequestImpl;
 import org.apache.hadoop.hbase.regionserver.compactions.DateTieredCompactionPolicy;
 import org.apache.hadoop.hbase.regionserver.compactions.DateTieredCompactionRequest;
 import org.apache.hadoop.hbase.regionserver.compactions.DateTieredCompactor;
 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
 import org.apache.hadoop.hbase.security.User;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * HBASE-15400 This store engine allows us to store data in date tiered layout with exponential
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreEngine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreEngine.java
index 58f8bbb..693b9c9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreEngine.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreEngine.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.regionserver;
 
 import java.io.IOException;
 import java.util.List;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.CellComparator;
@@ -39,8 +38,8 @@ import org.apache.yetus.audience.InterfaceAudience;
  * their derivatives.
  */
 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
-public class DefaultStoreEngine extends StoreEngine<
-  DefaultStoreFlusher, RatioBasedCompactionPolicy, DefaultCompactor, DefaultStoreFileManager> {
+public class DefaultStoreEngine extends StoreEngine<DefaultStoreFlusher,
+  RatioBasedCompactionPolicy, DefaultCompactor, DefaultStoreFileManager> {
 
   public static final String DEFAULT_STORE_FLUSHER_CLASS_KEY =
       "hbase.hstore.defaultengine.storeflusher.class";
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFlusher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFlusher.java
index a7d7fb1..306760d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFlusher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFlusher.java
@@ -21,15 +21,14 @@ package org.apache.hadoop.hbase.regionserver;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
-
-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.Path;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Default implementation of StoreFlusher.
@@ -60,9 +59,7 @@ public class DefaultStoreFlusher extends StoreFlusher {
       synchronized (flushLock) {
         status.setStatus("Flushing " + store + ": creating writer");
         // Write the map out to the disk
-        writer = store.createWriterInTmp(cellsCount,
-            store.getColumnFamilyDescriptor().getCompressionType(), false, true,
-            snapshot.isTagsPresent(), false);
+        writer = createWriter(snapshot, false);
         IOException e = null;
         try {
           performFlush(scanner, writer, throughputController);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java
index 3b50109..ce2b38b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java
@@ -27,7 +27,6 @@ import java.util.NavigableSet;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicLong;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -158,7 +157,7 @@ public class HMobStore extends HStore {
   protected StoreEngine<?, ?, ?, ?> createStoreEngine(HStore store, Configuration conf,
       CellComparator cellComparator) throws IOException {
     MobStoreEngine engine = new MobStoreEngine();
-    engine.createComponents(conf, store, cellComparator);
+    engine.createComponentsOnce(conf, store, cellComparator);
     return engine;
   }
 
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 667eabf..2f5f8d7 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
@@ -145,7 +145,7 @@ public class HRegionFileSystem {
   //  Temp Helpers
   // ===========================================================================
   /** @return {@link Path} to the region's temp directory, used for file creations */
-  Path getTempDir() {
+  public Path getTempDir() {
     return new Path(getRegionDir(), REGION_TEMP_DIR);
   }
 
@@ -240,11 +240,7 @@ public class HRegionFileSystem {
    * @param familyName Column Family Name
    * @return a set of {@link StoreFileInfo} for the specified family.
    */
-  public Collection<StoreFileInfo> getStoreFiles(final byte[] familyName) throws IOException {
-    return getStoreFiles(Bytes.toString(familyName));
-  }
-
-  public Collection<StoreFileInfo> getStoreFiles(final String familyName) throws IOException {
+  public List<StoreFileInfo> getStoreFiles(final String familyName) throws IOException {
     return getStoreFiles(familyName, true);
   }
 
@@ -254,7 +250,7 @@ public class HRegionFileSystem {
    * @param familyName Column Family Name
    * @return a set of {@link StoreFileInfo} for the specified family.
    */
-  public Collection<StoreFileInfo> getStoreFiles(final String familyName, final boolean validate)
+  public List<StoreFileInfo> getStoreFiles(final String familyName, final boolean validate)
       throws IOException {
     Path familyDir = getStoreDir(familyName);
     FileStatus[] files = CommonFSUtils.listStatus(this.fs, familyDir);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
index 0213827..5309305 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
+import com.google.errorprone.annotations.RestrictedApi;
 import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.net.InetSocketAddress;
@@ -47,8 +48,6 @@ import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.LongAdder;
 import java.util.concurrent.locks.ReentrantLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-import java.util.function.Predicate;
 import java.util.function.ToLongFunction;
 import java.util.stream.Collectors;
 import java.util.stream.LongStream;
@@ -70,17 +69,12 @@ import org.apache.hadoop.hbase.conf.ConfigurationManager;
 import org.apache.hadoop.hbase.conf.PropagatingConfigurationObserver;
 import org.apache.hadoop.hbase.coprocessor.ReadOnlyConfiguration;
 import org.apache.hadoop.hbase.io.HeapSize;
-import org.apache.hadoop.hbase.io.compress.Compression;
-import org.apache.hadoop.hbase.io.crypto.Encryption;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.HFile;
-import org.apache.hadoop.hbase.io.hfile.HFileContext;
-import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
 import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder;
 import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoderImpl;
 import org.apache.hadoop.hbase.io.hfile.HFileScanner;
 import org.apache.hadoop.hbase.io.hfile.InvalidHFileException;
-import org.apache.hadoop.hbase.log.HBaseMarkers;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
 import org.apache.hadoop.hbase.quotas.RegionSizeStore;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
@@ -110,7 +104,6 @@ import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableCollection
 import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList;
 import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
-import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
 import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
 import org.apache.hbase.thirdparty.org.apache.commons.collections4.IterableUtils;
 
@@ -166,16 +159,6 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
   private boolean cacheOnWriteLogged;
 
   /**
-   * RWLock for store operations.
-   * Locked in shared mode when the list of component stores is looked at:
-   *   - all reads/writes to table data
-   *   - checking for split
-   * Locked in exclusive mode when the list of component stores is modified:
-   *   - closing
-   *   - completing a compaction
-   */
-  final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
-  /**
    * Lock specific to archiving compacted store files.  This avoids races around
    * the combination of retrieving the list of compacted files and moving them to
    * the archive directory.  Since this is usually a background process (other than
@@ -283,14 +266,8 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
     }
 
     this.storeEngine = createStoreEngine(this, this.conf, region.getCellComparator());
-    List<HStoreFile> hStoreFiles = loadStoreFiles(warmup);
-    // Move the storeSize calculation out of loadStoreFiles() method, because the secondary read
-    // replica's refreshStoreFiles() will also use loadStoreFiles() to refresh its store files and
-    // update the storeSize in the refreshStoreSizeAndTotalBytes() finally (just like compaction) , so
-    // no need calculate the storeSize twice.
-    this.storeSize.addAndGet(getStorefilesSize(hStoreFiles, sf -> true));
-    this.totalUncompressedBytes.addAndGet(getTotalUncompressedBytes(hStoreFiles));
-    this.storeEngine.getStoreFileManager().loadFiles(hStoreFiles);
+    storeEngine.initialize(warmup);
+    refreshStoreSizeAndTotalBytes();
 
     flushRetriesNumber = conf.getInt(
         "hbase.hstore.flush.retries.number", DEFAULT_FLUSH_RETRIES_NUMBER);
@@ -510,105 +487,18 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
     this.dataBlockEncoder = blockEncoder;
   }
 
-  /**
-   * Creates an unsorted list of StoreFile loaded in parallel
-   * from the given directory.
-   */
-  private List<HStoreFile> loadStoreFiles(boolean warmup) throws IOException {
-    Collection<StoreFileInfo> files = getRegionFileSystem().getStoreFiles(getColumnFamilyName());
-    return openStoreFiles(files, warmup);
-  }
-
-  private List<HStoreFile> openStoreFiles(Collection<StoreFileInfo> files, boolean warmup)
-      throws IOException {
-    if (CollectionUtils.isEmpty(files)) {
-      return Collections.emptyList();
-    }
-    // initialize the thread pool for opening store files in parallel..
-    ThreadPoolExecutor storeFileOpenerThreadPool =
-      this.region.getStoreFileOpenAndCloseThreadPool("StoreFileOpener-" +
-          this.getColumnFamilyName());
-    CompletionService<HStoreFile> completionService =
-      new ExecutorCompletionService<>(storeFileOpenerThreadPool);
-
-    int totalValidStoreFile = 0;
-    for (StoreFileInfo storeFileInfo : files) {
-      // The StoreFileInfo will carry store configuration down to HFile, we need to set it to
-      // our store's CompoundConfiguration here.
-      storeFileInfo.setConf(conf);
-      // open each store file in parallel
-      completionService.submit(() -> this.createStoreFileAndReader(storeFileInfo));
-      totalValidStoreFile++;
-    }
-
-    Set<String> compactedStoreFiles = new HashSet<>();
-    ArrayList<HStoreFile> results = new ArrayList<>(files.size());
-    IOException ioe = null;
-    try {
-      for (int i = 0; i < totalValidStoreFile; i++) {
-        try {
-          HStoreFile storeFile = completionService.take().get();
-          if (storeFile != null) {
-            LOG.debug("loaded {}", storeFile);
-            results.add(storeFile);
-            compactedStoreFiles.addAll(storeFile.getCompactedStoreFiles());
-          }
-        } catch (InterruptedException e) {
-          if (ioe == null) {
-            ioe = new InterruptedIOException(e.getMessage());
-          }
-        } catch (ExecutionException e) {
-          if (ioe == null) {
-            ioe = new IOException(e.getCause());
-          }
-        }
-      }
-    } finally {
-      storeFileOpenerThreadPool.shutdownNow();
-    }
-    if (ioe != null) {
-      // close StoreFile readers
-      boolean evictOnClose =
-          getCacheConfig() != null? getCacheConfig().shouldEvictOnClose(): true;
-      for (HStoreFile file : results) {
-        try {
-          if (file != null) {
-            file.closeStoreFile(evictOnClose);
-          }
-        } catch (IOException e) {
-          LOG.warn("Could not close store file {}", file, e);
-        }
-      }
-      throw ioe;
-    }
-
-    // Should not archive the compacted store files when region warmup. See HBASE-22163.
-    if (!warmup) {
-      // Remove the compacted files from result
-      List<HStoreFile> filesToRemove = new ArrayList<>(compactedStoreFiles.size());
-      for (HStoreFile storeFile : results) {
-        if (compactedStoreFiles.contains(storeFile.getPath().getName())) {
-          LOG.warn("Clearing the compacted storefile {} from {}", storeFile, this);
-          storeFile.getReader().close(storeFile.getCacheConf() != null ?
-            storeFile.getCacheConf().shouldEvictOnClose() : true);
-          filesToRemove.add(storeFile);
-        }
-      }
-      results.removeAll(filesToRemove);
-      if (!filesToRemove.isEmpty() && this.isPrimaryReplicaStore()) {
-        LOG.debug("Moving the files {} to archive", filesToRemove);
-        getRegionFileSystem().removeStoreFiles(this.getColumnFamilyDescriptor().getNameAsString(),
-            filesToRemove);
-      }
-    }
-
-    return results;
+  private void postRefreshStoreFiles() throws IOException {
+    // Advance the memstore read point to be at least the new store files seqIds so that
+    // readers might pick it up. This assumes that the store is not getting any writes (otherwise
+    // in-flight transactions might be made visible)
+    getMaxSequenceId().ifPresent(region.getMVCC()::advanceTo);
+    refreshStoreSizeAndTotalBytes();
   }
 
   @Override
   public void refreshStoreFiles() throws IOException {
-    Collection<StoreFileInfo> newFiles = getRegionFileSystem().getStoreFiles(getColumnFamilyName());
-    refreshStoreFilesInternal(newFiles);
+    storeEngine.refreshStoreFiles();
+    postRefreshStoreFiles();
   }
 
   /**
@@ -616,89 +506,8 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
    * region replicas to keep up to date with the primary region files.
    */
   public void refreshStoreFiles(Collection<String> newFiles) throws IOException {
-    List<StoreFileInfo> storeFiles = new ArrayList<>(newFiles.size());
-    for (String file : newFiles) {
-      storeFiles.add(getRegionFileSystem().getStoreFileInfo(getColumnFamilyName(), file));
-    }
-    refreshStoreFilesInternal(storeFiles);
-  }
-
-  /**
-   * Checks the underlying store files, and opens the files that  have not
-   * been opened, and removes the store file readers for store files no longer
-   * available. Mainly used by secondary region replicas to keep up to date with
-   * the primary region files.
-   */
-  private void refreshStoreFilesInternal(Collection<StoreFileInfo> newFiles) throws IOException {
-    StoreFileManager sfm = storeEngine.getStoreFileManager();
-    Collection<HStoreFile> currentFiles = sfm.getStorefiles();
-    Collection<HStoreFile> compactedFiles = sfm.getCompactedfiles();
-    if (currentFiles == null) {
-      currentFiles = Collections.emptySet();
-    }
-    if (newFiles == null) {
-      newFiles = Collections.emptySet();
-    }
-    if (compactedFiles == null) {
-      compactedFiles = Collections.emptySet();
-    }
-
-    HashMap<StoreFileInfo, HStoreFile> currentFilesSet = new HashMap<>(currentFiles.size());
-    for (HStoreFile sf : currentFiles) {
-      currentFilesSet.put(sf.getFileInfo(), sf);
-    }
-    HashMap<StoreFileInfo, HStoreFile> compactedFilesSet = new HashMap<>(compactedFiles.size());
-    for (HStoreFile sf : compactedFiles) {
-      compactedFilesSet.put(sf.getFileInfo(), sf);
-    }
-
-    Set<StoreFileInfo> newFilesSet = new HashSet<StoreFileInfo>(newFiles);
-    // Exclude the files that have already been compacted
-    newFilesSet = Sets.difference(newFilesSet, compactedFilesSet.keySet());
-    Set<StoreFileInfo> toBeAddedFiles = Sets.difference(newFilesSet, currentFilesSet.keySet());
-    Set<StoreFileInfo> toBeRemovedFiles = Sets.difference(currentFilesSet.keySet(), newFilesSet);
-
-    if (toBeAddedFiles.isEmpty() && toBeRemovedFiles.isEmpty()) {
-      return;
-    }
-
-    LOG.info("Refreshing store files for " + this + " files to add: "
-      + toBeAddedFiles + " files to remove: " + toBeRemovedFiles);
-
-    Set<HStoreFile> toBeRemovedStoreFiles = new HashSet<>(toBeRemovedFiles.size());
-    for (StoreFileInfo sfi : toBeRemovedFiles) {
-      toBeRemovedStoreFiles.add(currentFilesSet.get(sfi));
-    }
-
-    // try to open the files
-    List<HStoreFile> openedFiles = openStoreFiles(toBeAddedFiles, false);
-
-    // propogate the file changes to the underlying store file manager
-    replaceStoreFiles(toBeRemovedStoreFiles, openedFiles); //won't throw an exception
-
-    // Advance the memstore read point to be at least the new store files seqIds so that
-    // readers might pick it up. This assumes that the store is not getting any writes (otherwise
-    // in-flight transactions might be made visible)
-    if (!toBeAddedFiles.isEmpty()) {
-      // we must have the max sequence id here as we do have several store files
-      region.getMVCC().advanceTo(this.getMaxSequenceId().getAsLong());
-    }
-
-    refreshStoreSizeAndTotalBytes();
-  }
-
-  protected HStoreFile createStoreFileAndReader(final Path p) throws IOException {
-    StoreFileInfo info = new StoreFileInfo(conf, this.getFileSystem(),
-        p, isPrimaryReplicaStore());
-    return createStoreFileAndReader(info);
-  }
-
-  private HStoreFile createStoreFileAndReader(StoreFileInfo info) throws IOException {
-    info.setRegionCoprocessorHost(this.region.getCoprocessorHost());
-    HStoreFile storeFile = new HStoreFile(info, getColumnFamilyDescriptor().getBloomFilterType(),
-            getCacheConfig());
-    storeFile.initReader();
-    return storeFile;
+    storeEngine.refreshStoreFiles(newFiles);
+    postRefreshStoreFiles();
   }
 
   /**
@@ -721,7 +530,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
    * Adds a value to the memstore
    */
   public void add(final Cell cell, MemStoreSizing memstoreSizing) {
-    lock.readLock().lock();
+    storeEngine.readLock();
     try {
       if (this.currentParallelPutCount.getAndIncrement() > this.parallelPutCountPrintThreshold) {
         LOG.trace("tableName={}, encodedName={}, columnFamilyName={} is too busy!",
@@ -729,7 +538,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
       }
       this.memstore.add(cell, memstoreSizing);
     } finally {
-      lock.readLock().unlock();
+      storeEngine.readUnlock();
       currentParallelPutCount.decrementAndGet();
     }
   }
@@ -738,7 +547,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
    * Adds the specified value to the memstore
    */
   public void add(final Iterable<Cell> cells, MemStoreSizing memstoreSizing) {
-    lock.readLock().lock();
+    storeEngine.readLock();
     try {
       if (this.currentParallelPutCount.getAndIncrement() > this.parallelPutCountPrintThreshold) {
         LOG.trace("tableName={}, encodedName={}, columnFamilyName={} is too busy!",
@@ -746,7 +555,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
       }
       memstore.add(cells, memstoreSizing);
     } finally {
-      lock.readLock().unlock();
+      storeEngine.readUnlock();
       currentParallelPutCount.decrementAndGet();
     }
   }
@@ -869,17 +678,16 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
     LOG.info("Loaded HFile " + srcPath + " into " + this + " as "
         + dstPath + " - updating store file list.");
 
-    HStoreFile sf = createStoreFileAndReader(dstPath);
+    HStoreFile sf = storeEngine.createStoreFileAndReader(dstPath);
     bulkLoadHFile(sf);
 
-    LOG.info("Successfully loaded {} into {} (new location: {})",
-        srcPath, this, dstPath);
+    LOG.info("Successfully loaded {} into {} (new location: {})", srcPath, this, dstPath);
 
     return dstPath;
   }
 
   public void bulkLoadHFile(StoreFileInfo fileInfo) throws IOException {
-    HStoreFile sf = createStoreFileAndReader(fileInfo);
+    HStoreFile sf = storeEngine.createStoreFileAndReader(fileInfo);
     bulkLoadHFile(sf);
   }
 
@@ -887,28 +695,75 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
     StoreFileReader r = sf.getReader();
     this.storeSize.addAndGet(r.length());
     this.totalUncompressedBytes.addAndGet(r.getTotalUncompressedBytes());
-
-    // Append the new storefile into the list
-    this.lock.writeLock().lock();
-    try {
-      this.storeEngine.getStoreFileManager().insertNewFiles(Lists.newArrayList(sf));
-    } finally {
-      // We need the lock, as long as we are updating the storeFiles
-      // or changing the memstore. Let us release it before calling
-      // notifyChangeReadersObservers. See HBASE-4485 for a possible
-      // deadlock scenario that could have happened if continue to hold
-      // the lock.
-      this.lock.writeLock().unlock();
-    }
+    storeEngine.addStoreFiles(Lists.newArrayList(sf), () -> {
+    });
     LOG.info("Loaded HFile " + sf.getFileInfo() + " into " + this);
     if (LOG.isTraceEnabled()) {
-      String traceMessage = "BULK LOAD time,size,store size,store files ["
-          + EnvironmentEdgeManager.currentTime() + "," + r.length() + "," + storeSize
-          + "," + storeEngine.getStoreFileManager().getStorefileCount() + "]";
+      String traceMessage = "BULK LOAD time,size,store size,store files [" +
+        EnvironmentEdgeManager.currentTime() + "," + r.length() + "," + storeSize + "," +
+        storeEngine.getStoreFileManager().getStorefileCount() + "]";
       LOG.trace(traceMessage);
     }
   }
 
+  private ImmutableCollection<HStoreFile> closeWithoutLock() throws IOException {
+    // Clear so metrics doesn't find them.
+    ImmutableCollection<HStoreFile> result = storeEngine.getStoreFileManager().clearFiles();
+    Collection<HStoreFile> compactedfiles = storeEngine.getStoreFileManager().clearCompactedFiles();
+    // clear the compacted files
+    if (CollectionUtils.isNotEmpty(compactedfiles)) {
+      removeCompactedfiles(compactedfiles,
+        getCacheConfig() != null ? getCacheConfig().shouldEvictOnClose() : true);
+    }
+    if (!result.isEmpty()) {
+      // initialize the thread pool for closing store files in parallel.
+      ThreadPoolExecutor storeFileCloserThreadPool =
+        this.region.getStoreFileOpenAndCloseThreadPool("StoreFileCloser-" +
+          this.region.getRegionInfo().getEncodedName() + "-" + this.getColumnFamilyName());
+
+      // close each store file in parallel
+      CompletionService<Void> completionService =
+        new ExecutorCompletionService<>(storeFileCloserThreadPool);
+      for (HStoreFile f : result) {
+        completionService.submit(new Callable<Void>() {
+          @Override
+          public Void call() throws IOException {
+            boolean evictOnClose =
+              getCacheConfig() != null ? getCacheConfig().shouldEvictOnClose() : true;
+            f.closeStoreFile(evictOnClose);
+            return null;
+          }
+        });
+      }
+
+      IOException ioe = null;
+      try {
+        for (int i = 0; i < result.size(); i++) {
+          try {
+            Future<Void> future = completionService.take();
+            future.get();
+          } catch (InterruptedException e) {
+            if (ioe == null) {
+              ioe = new InterruptedIOException();
+              ioe.initCause(e);
+            }
+          } catch (ExecutionException e) {
+            if (ioe == null) {
+              ioe = new IOException(e.getCause());
+            }
+          }
+        }
+      } finally {
+        storeFileCloserThreadPool.shutdownNow();
+      }
+      if (ioe != null) {
+        throw ioe;
+      }
+    }
+    LOG.trace("Closed {}", this);
+    return result;
+  }
+
   /**
    * Close all the readers We don't need to worry about subsequent requests because the Region holds
    * a write lock that will prevent any more reads or writes.
@@ -916,67 +771,18 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
    * @throws IOException on failure
    */
   public ImmutableCollection<HStoreFile> close() throws IOException {
+    // findbugs can not recognize storeEngine.writeLock is just a lock operation so it will report
+    // UL_UNRELEASED_LOCK_EXCEPTION_PATH, so here we have to use two try finally...
+    // Change later if findbugs becomes smarter in the future.
     this.archiveLock.lock();
-    this.lock.writeLock().lock();
     try {
-      // Clear so metrics doesn't find them.
-      ImmutableCollection<HStoreFile> result = storeEngine.getStoreFileManager().clearFiles();
-      Collection<HStoreFile> compactedfiles =
-          storeEngine.getStoreFileManager().clearCompactedFiles();
-      // clear the compacted files
-      if (CollectionUtils.isNotEmpty(compactedfiles)) {
-        removeCompactedfiles(compactedfiles, getCacheConfig() != null ?
-            getCacheConfig().shouldEvictOnClose() : true);
-      }
-      if (!result.isEmpty()) {
-        // initialize the thread pool for closing store files in parallel.
-        ThreadPoolExecutor storeFileCloserThreadPool = this.region
-            .getStoreFileOpenAndCloseThreadPool("StoreFileCloser-"
-                + this.getColumnFamilyName());
-
-        // close each store file in parallel
-        CompletionService<Void> completionService =
-          new ExecutorCompletionService<>(storeFileCloserThreadPool);
-        for (HStoreFile f : result) {
-          completionService.submit(new Callable<Void>() {
-            @Override
-            public Void call() throws IOException {
-              boolean evictOnClose =
-                  getCacheConfig() != null? getCacheConfig().shouldEvictOnClose(): true;
-              f.closeStoreFile(evictOnClose);
-              return null;
-            }
-          });
-        }
-
-        IOException ioe = null;
-        try {
-          for (int i = 0; i < result.size(); i++) {
-            try {
-              Future<Void> future = completionService.take();
-              future.get();
-            } catch (InterruptedException e) {
-              if (ioe == null) {
-                ioe = new InterruptedIOException();
-                ioe.initCause(e);
-              }
-            } catch (ExecutionException e) {
-              if (ioe == null) {
-                ioe = new IOException(e.getCause());
-              }
-            }
-          }
-        } finally {
-          storeFileCloserThreadPool.shutdownNow();
-        }
-        if (ioe != null) {
-          throw ioe;
-        }
+      this.storeEngine.writeLock();
+      try {
+        return closeWithoutLock();
+      } finally {
+        this.storeEngine.writeUnlock();
       }
-      LOG.trace("Closed {}", this);
-      return result;
     } finally {
-      this.lock.writeLock().unlock();
       this.archiveLock.unlock();
     }
   }
@@ -1006,7 +812,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
         try {
           for (Path pathName : pathNames) {
             lastPathName = pathName;
-            validateStoreFile(pathName);
+            storeEngine.validateStoreFile(pathName);
           }
           return pathNames;
         } catch (Exception e) {
@@ -1052,204 +858,37 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
     }
 
     Path dstPath = getRegionFileSystem().commitStoreFile(getColumnFamilyName(), path);
-    HStoreFile sf = createStoreFileAndReader(dstPath);
+    HStoreFile sf = storeEngine.createStoreFileAndReader(dstPath);
     StoreFileReader r = sf.getReader();
     this.storeSize.addAndGet(r.length());
     this.totalUncompressedBytes.addAndGet(r.getTotalUncompressedBytes());
 
-    this.lock.writeLock().lock();
-    try {
-      this.storeEngine.getStoreFileManager().insertNewFiles(Lists.newArrayList(sf));
-    } finally {
-      this.lock.writeLock().unlock();
-    }
+    storeEngine.addStoreFiles(Lists.newArrayList(sf), () -> {
+    });
 
     LOG.info("Loaded recovered hfile to {}, entries={}, sequenceid={}, filesize={}", sf,
       r.getEntries(), r.getSequenceID(), TraditionalBinaryPrefix.long2String(r.length(), "B", 1));
     return sf;
   }
 
-  /**
-   * Commit the given {@code files}.
-   * <p/>
-   * We will move the file into data directory, and open it.
-   * @param files the files want to commit
-   * @param validate whether to validate the store files
-   * @return the committed store files
-   */
-  private List<HStoreFile> commitStoreFiles(List<Path> files, boolean validate) throws IOException {
-    List<HStoreFile> committedFiles = new ArrayList<>(files.size());
-    HRegionFileSystem hfs = getRegionFileSystem();
-    String familyName = getColumnFamilyName();
-    for (Path file : files) {
-      try {
-        if (validate) {
-          validateStoreFile(file);
-        }
-        Path committedPath = hfs.commitStoreFile(familyName, file);
-        HStoreFile sf = createStoreFileAndReader(committedPath);
-        committedFiles.add(sf);
-      } catch (IOException e) {
-        LOG.error("Failed to commit store file {}", file, e);
-        // Try to delete the files we have committed before.
-        // It is OK to fail when deleting as leaving the file there does not cause any data
-        // corruption problem. It just introduces some duplicated data which may impact read
-        // performance a little when reading before compaction.
-        for (HStoreFile sf : committedFiles) {
-          Path pathToDelete = sf.getPath();
-          try {
-            sf.deleteStoreFile();
-          } catch (IOException deleteEx) {
-            LOG.warn(HBaseMarkers.FATAL, "Failed to delete committed store file {}", pathToDelete,
-              deleteEx);
-          }
-        }
-        throw new IOException("Failed to commit the flush", e);
-      }
-    }
-    return committedFiles;
-  }
-
-  public StoreFileWriter createWriterInTmp(long maxKeyCount, Compression.Algorithm compression,
-    boolean isCompaction, boolean includeMVCCReadpoint, boolean includesTag,
-    boolean shouldDropBehind) throws IOException {
-    return createWriterInTmp(maxKeyCount, compression, isCompaction, includeMVCCReadpoint,
-      includesTag, shouldDropBehind, -1, HConstants.EMPTY_STRING);
-  }
-
-  /**
-   * @param compression Compression algorithm to use
-   * @param isCompaction whether we are creating a new file in a compaction
-   * @param includeMVCCReadpoint - whether to include MVCC or not
-   * @param includesTag - includesTag or not
-   * @return Writer for a new StoreFile in the tmp dir.
-   */
-  // TODO : allow the Writer factory to create Writers of ShipperListener type only in case of
-  // compaction
-  public StoreFileWriter createWriterInTmp(long maxKeyCount, Compression.Algorithm compression,
-      boolean isCompaction, boolean includeMVCCReadpoint, boolean includesTag,
-      boolean shouldDropBehind, long totalCompactedFilesSize, String fileStoragePolicy)
-        throws IOException {
-    // creating new cache config for each new writer
-    final CacheConfig cacheConf = getCacheConfig();
-    final CacheConfig writerCacheConf = new CacheConfig(cacheConf);
-    if (isCompaction) {
-      // Don't cache data on write on compactions, unless specifically configured to do so
-      // Cache only when total file size remains lower than configured threshold
-      final boolean cacheCompactedBlocksOnWrite =
-        getCacheConfig().shouldCacheCompactedBlocksOnWrite();
-      // if data blocks are to be cached on write
-      // during compaction, we should forcefully
-      // cache index and bloom blocks as well
-      if (cacheCompactedBlocksOnWrite && totalCompactedFilesSize <= cacheConf
-        .getCacheCompactedBlocksOnWriteThreshold()) {
-        writerCacheConf.enableCacheOnWrite();
-        if (!cacheOnWriteLogged) {
-          LOG.info("For {} , cacheCompactedBlocksOnWrite is true, hence enabled " +
-              "cacheOnWrite for Data blocks, Index blocks and Bloom filter blocks", this);
-          cacheOnWriteLogged = true;
-        }
-      } else {
-        writerCacheConf.setCacheDataOnWrite(false);
-        if (totalCompactedFilesSize > cacheConf.getCacheCompactedBlocksOnWriteThreshold()) {
-          // checking condition once again for logging
-          LOG.debug(
-            "For {}, setting cacheCompactedBlocksOnWrite as false as total size of compacted "
-              + "files - {}, is greater than cacheCompactedBlocksOnWriteThreshold - {}",
-            this, totalCompactedFilesSize,
-            cacheConf.getCacheCompactedBlocksOnWriteThreshold());
-        }
-      }
-    } else {
-      final boolean shouldCacheDataOnWrite = cacheConf.shouldCacheDataOnWrite();
-      if (shouldCacheDataOnWrite) {
-        writerCacheConf.enableCacheOnWrite();
-        if (!cacheOnWriteLogged) {
-          LOG.info("For {} , cacheDataOnWrite is true, hence enabled cacheOnWrite for " +
-            "Index blocks and Bloom filter blocks", this);
-          cacheOnWriteLogged = true;
-        }
-      }
-    }
-    Encryption.Context encryptionContext = storeContext.getEncryptionContext();
-    HFileContext hFileContext = createFileContext(compression, includeMVCCReadpoint, includesTag,
-      encryptionContext);
-    Path familyTempDir = new Path(getRegionFileSystem().getTempDir(), getColumnFamilyName());
-    StoreFileWriter.Builder builder =
-      new StoreFileWriter.Builder(conf, writerCacheConf, getFileSystem())
-        .withOutputDir(familyTempDir)
-        .withBloomType(storeContext.getBloomFilterType())
-        .withMaxKeyCount(maxKeyCount)
-        .withFavoredNodes(storeContext.getFavoredNodes())
-        .withFileContext(hFileContext)
-        .withShouldDropCacheBehind(shouldDropBehind)
-        .withCompactedFilesSupplier(storeContext.getCompactedFilesSupplier())
-        .withFileStoragePolicy(fileStoragePolicy);
-    return builder.build();
-  }
-
-  private HFileContext createFileContext(Compression.Algorithm compression,
-    boolean includeMVCCReadpoint, boolean includesTag, Encryption.Context encryptionContext) {
-    if (compression == null) {
-      compression = HFile.DEFAULT_COMPRESSION_ALGORITHM;
-    }
-    ColumnFamilyDescriptor family = getColumnFamilyDescriptor();
-    HFileContext hFileContext = new HFileContextBuilder()
-      .withIncludesMvcc(includeMVCCReadpoint)
-      .withIncludesTags(includesTag)
-      .withCompression(compression)
-      .withCompressTags(family.isCompressTags())
-      .withChecksumType(StoreUtils.getChecksumType(conf))
-      .withBytesPerCheckSum(StoreUtils.getBytesPerChecksum(conf))
-      .withBlockSize(family.getBlocksize())
-      .withHBaseCheckSum(true)
-      .withDataBlockEncoding(family.getDataBlockEncoding())
-      .withEncryptionContext(encryptionContext)
-      .withCreateTime(EnvironmentEdgeManager.currentTime())
-      .withColumnFamily(getColumnFamilyDescriptor().getName())
-      .withTableName(getTableName().getName())
-      .withCellComparator(getComparator())
-      .build();
-    return hFileContext;
-  }
-
   private long getTotalSize(Collection<HStoreFile> sfs) {
     return sfs.stream().mapToLong(sf -> sf.getReader().length()).sum();
   }
 
-  /**
-   * Change storeFiles adding into place the Reader produced by this new flush.
-   * @param sfs Store files
-   * @return Whether compaction is required.
-   */
-  private boolean updateStorefiles(List<HStoreFile> sfs, long snapshotId) throws IOException {
-    this.lock.writeLock().lock();
-    try {
-      this.storeEngine.getStoreFileManager().insertNewFiles(sfs);
-      /**
-       * NOTE:we should keep clearSnapshot method inside the write lock because clearSnapshot may
-       * close {@link DefaultMemStore#snapshot}, which may be used by
-       * {@link DefaultMemStore#getScanners}.
-       */
-      if (snapshotId > 0) {
-        this.memstore.clearSnapshot(snapshotId);
-      }
-    } finally {
-      // We need the lock, as long as we are updating the storeFiles
-      // or changing the memstore. Let us release it before calling
-      // notifyChangeReadersObservers. See HBASE-4485 for a possible
-      // deadlock scenario that could have happened if continue to hold
-      // the lock.
-      this.lock.writeLock().unlock();
-    }
-
+  private boolean completeFlush(List<HStoreFile> sfs, long snapshotId) throws IOException {
+    // NOTE:we should keep clearSnapshot method inside the write lock because clearSnapshot may
+    // close {@link DefaultMemStore#snapshot}, which may be used by
+    // {@link DefaultMemStore#getScanners}.
+    storeEngine.addStoreFiles(sfs,
+      snapshotId > 0 ? () -> this.memstore.clearSnapshot(snapshotId) : () -> {
+      });
     // notify to be called here - only in case of flushes
     notifyChangedReadersObservers(sfs);
     if (LOG.isTraceEnabled()) {
       long totalSize = getTotalSize(sfs);
-      String traceMessage = "FLUSH time,count,size,store size,store files ["
-          + EnvironmentEdgeManager.currentTime() + "," + sfs.size() + "," + totalSize
-          + "," + storeSize + "," + storeEngine.getStoreFileManager().getStorefileCount() + "]";
+      String traceMessage = "FLUSH time,count,size,store size,store files [" +
+        EnvironmentEdgeManager.currentTime() + "," + sfs.size() + "," + totalSize + "," +
+        storeSize + "," + storeEngine.getStoreFileManager().getStorefileCount() + "]";
       LOG.trace(traceMessage);
     }
     return needsCompaction();
@@ -1261,11 +900,11 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
   private void notifyChangedReadersObservers(List<HStoreFile> sfs) throws IOException {
     for (ChangedReadersObserver o : this.changedReaderObservers) {
       List<KeyValueScanner> memStoreScanners;
-      this.lock.readLock().lock();
+      this.storeEngine.readLock();
       try {
         memStoreScanners = this.memstore.getScanners(o.getReadPoint());
       } finally {
-        this.lock.readLock().unlock();
+        this.storeEngine.readUnlock();
       }
       o.updateReaders(sfs, memStoreScanners);
     }
@@ -1307,13 +946,13 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
       byte[] stopRow, boolean includeStopRow, long readPt) throws IOException {
     Collection<HStoreFile> storeFilesToScan;
     List<KeyValueScanner> memStoreScanners;
-    this.lock.readLock().lock();
+    this.storeEngine.readLock();
     try {
       storeFilesToScan = this.storeEngine.getStoreFileManager().getFilesForScan(startRow,
         includeStartRow, stopRow, includeStopRow);
       memStoreScanners = this.memstore.getScanners(readPt);
     } finally {
-      this.lock.readLock().unlock();
+      this.storeEngine.readUnlock();
     }
 
     try {
@@ -1390,11 +1029,11 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
       boolean includeMemstoreScanner) throws IOException {
     List<KeyValueScanner> memStoreScanners = null;
     if (includeMemstoreScanner) {
-      this.lock.readLock().lock();
+      this.storeEngine.readLock();
       try {
         memStoreScanners = this.memstore.getScanners(readPt);
       } finally {
-        this.lock.readLock().unlock();
+        this.storeEngine.readUnlock();
       }
     }
     try {
@@ -1510,14 +1149,13 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
       List<Path> newFiles) throws IOException {
     // Do the steps necessary to complete the compaction.
     setStoragePolicyFromFileName(newFiles);
-    List<HStoreFile> sfs = commitStoreFiles(newFiles, true);
+    List<HStoreFile> sfs = storeEngine.commitStoreFiles(newFiles, true);
     if (this.getCoprocessorHost() != null) {
       for (HStoreFile sf : sfs) {
         getCoprocessorHost().postCompact(this, sf, cr.getTracker(), cr, user);
       }
     }
-    writeCompactionWalRecord(filesToCompact, sfs);
-    replaceStoreFiles(filesToCompact, sfs);
+    replaceStoreFiles(filesToCompact, sfs, true);
     if (cr.isMajor()) {
       majorCompactedCellsCount.addAndGet(getCompactionProgress().getTotalCompactingKVs());
       majorCompactedCellsSize.addAndGet(getCompactionProgress().totalCompactedSize);
@@ -1581,25 +1219,24 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
         this.region.getRegionInfo(), compactionDescriptor, this.region.getMVCC());
   }
 
-  void replaceStoreFiles(Collection<HStoreFile> compactedFiles, Collection<HStoreFile> result)
-      throws IOException {
-    this.lock.writeLock().lock();
-    try {
-      this.storeEngine.getStoreFileManager().addCompactionResults(compactedFiles, result);
-      synchronized (filesCompacting) {
-        filesCompacting.removeAll(compactedFiles);
-      }
-
-      // These may be null when the RS is shutting down. The space quota Chores will fix the Region
-      // sizes later so it's not super-critical if we miss these.
-      RegionServerServices rsServices = region.getRegionServerServices();
-      if (rsServices != null && rsServices.getRegionServerSpaceQuotaManager() != null) {
-        updateSpaceQuotaAfterFileReplacement(
-            rsServices.getRegionServerSpaceQuotaManager().getRegionSizeStore(), getRegionInfo(),
-            compactedFiles, result);
-      }
-    } finally {
-      this.lock.writeLock().unlock();
+  @RestrictedApi(explanation = "Should only be called in TestHStore", link = "",
+    allowedOnPath = ".*/(HStore|TestHStore).java")
+  void replaceStoreFiles(Collection<HStoreFile> compactedFiles, Collection<HStoreFile> result,
+    boolean writeCompactionMarker) throws IOException {
+    storeEngine.replaceStoreFiles(compactedFiles, result);
+    if (writeCompactionMarker) {
+      writeCompactionWalRecord(compactedFiles, result);
+    }
+    synchronized (filesCompacting) {
+      filesCompacting.removeAll(compactedFiles);
+    }
+    // These may be null when the RS is shutting down. The space quota Chores will fix the Region
+    // sizes later so it's not super-critical if we miss these.
+    RegionServerServices rsServices = region.getRegionServerServices();
+    if (rsServices != null && rsServices.getRegionServerSpaceQuotaManager() != null) {
+      updateSpaceQuotaAfterFileReplacement(
+        rsServices.getRegionServerSpaceQuotaManager().getRegionSizeStore(), getRegionInfo(),
+        compactedFiles, result);
     }
   }
 
@@ -1722,7 +1359,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
       for (String compactionOutput : compactionOutputs) {
         StoreFileInfo storeFileInfo =
             getRegionFileSystem().getStoreFileInfo(getColumnFamilyName(), compactionOutput);
-        HStoreFile storeFile = createStoreFileAndReader(storeFileInfo);
+        HStoreFile storeFile = storeEngine.createStoreFileAndReader(storeFileInfo);
         outputStoreFiles.add(storeFile);
       }
     }
@@ -1730,7 +1367,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
     if (!inputStoreFiles.isEmpty() || !outputStoreFiles.isEmpty()) {
       LOG.info("Replaying compaction marker, replacing input files: " +
           inputStoreFiles + " with output files : " + outputStoreFiles);
-      this.replaceStoreFiles(inputStoreFiles, outputStoreFiles);
+      this.replaceStoreFiles(inputStoreFiles, outputStoreFiles, false);
       this.refreshStoreSizeAndTotalBytes();
     }
   }
@@ -1739,14 +1376,14 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
   public boolean hasReferences() {
     // Grab the read lock here, because we need to ensure that: only when the atomic
     // replaceStoreFiles(..) finished, we can get all the complete store file list.
-    this.lock.readLock().lock();
+    this.storeEngine.readLock();
     try {
       // Merge the current store files with compacted files here due to HBASE-20940.
       Collection<HStoreFile> allStoreFiles = new ArrayList<>(getStorefiles());
       allStoreFiles.addAll(getCompactedFiles());
       return StoreUtils.hasReferences(allStoreFiles);
     } finally {
-      this.lock.readLock().unlock();
+      this.storeEngine.readUnlock();
     }
   }
 
@@ -1786,7 +1423,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
 
     final CompactionContext compaction = storeEngine.createCompaction();
     CompactionRequestImpl request = null;
-    this.lock.readLock().lock();
+    this.storeEngine.readLock();
     try {
       synchronized (filesCompacting) {
         // First, see if coprocessor would want to override selection.
@@ -1859,7 +1496,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
         request.setTracker(tracker);
       }
     } finally {
-      this.lock.readLock().unlock();
+      this.storeEngine.readUnlock();
     }
 
     if (LOG.isDebugEnabled()) {
@@ -1892,7 +1529,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
           this, getColumnFamilyDescriptor().getMinVersions());
       return;
     }
-    this.lock.readLock().lock();
+    this.storeEngine.readLock();
     Collection<HStoreFile> delSfs = null;
     try {
       synchronized (filesCompacting) {
@@ -1904,7 +1541,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
         }
       }
     } finally {
-      this.lock.readLock().unlock();
+      this.storeEngine.readUnlock();
     }
 
     if (CollectionUtils.isEmpty(delSfs)) {
@@ -1912,8 +1549,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
     }
 
     Collection<HStoreFile> newFiles = Collections.emptyList(); // No new files.
-    writeCompactionWalRecord(delSfs, newFiles);
-    replaceStoreFiles(delSfs, newFiles);
+    replaceStoreFiles(delSfs, newFiles, true);
     refreshStoreSizeAndTotalBytes();
     LOG.info("Completed removal of " + delSfs.size() + " unnecessary (expired) file(s) in "
         + this + "; total size is "
@@ -1936,25 +1572,6 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
   }
 
   /**
-   * Validates a store file by opening and closing it. In HFileV2 this should not be an expensive
-   * operation.
-   * @param path the path to the store file
-   */
-  private void validateStoreFile(Path path) throws IOException {
-    HStoreFile storeFile = null;
-    try {
-      storeFile = createStoreFileAndReader(path);
-    } catch (IOException e) {
-      LOG.error("Failed to open store file : {}, keeping it in tmp location", path, e);
-      throw e;
-    } finally {
-      if (storeFile != null) {
-        storeFile.closeStoreFile(false);
-      }
-    }
-  }
-
-  /**
    * Update counts.
    */
   protected void refreshStoreSizeAndTotalBytes()
@@ -1999,7 +1616,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
    * Determines if Store should be split.
    */
   public Optional<byte[]> getSplitPoint() {
-    this.lock.readLock().lock();
+    this.storeEngine.readLock();
     try {
       // Should already be enforced by the split policy!
       assert !this.getRegionInfo().isMetaRegion();
@@ -2012,7 +1629,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
     } catch(IOException e) {
       LOG.warn("Failed getting store size for {}", this, e);
     } finally {
-      this.lock.readLock().unlock();
+      this.storeEngine.readUnlock();
     }
     return Optional.empty();
   }
@@ -2045,7 +1662,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
    */
   public KeyValueScanner getScanner(Scan scan, final NavigableSet<byte[]> targetCols, long readPt)
       throws IOException {
-    lock.readLock().lock();
+    storeEngine.readLock();
     try {
       ScanInfo scanInfo;
       if (this.getCoprocessorHost() != null) {
@@ -2055,7 +1672,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
       }
       return createScanner(scan, scanInfo, targetCols, readPt);
     } finally {
-      lock.readLock().unlock();
+      storeEngine.readUnlock();
     }
   }
 
@@ -2085,7 +1702,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
       boolean cacheBlocks, boolean usePread, boolean isCompaction, ScanQueryMatcher matcher,
       byte[] startRow, boolean includeStartRow, byte[] stopRow, boolean includeStopRow, long readPt,
       boolean includeMemstoreScanner) throws IOException {
-    this.lock.readLock().lock();
+    this.storeEngine.readLock();
     try {
       Map<String, HStoreFile> name2File =
           new HashMap<>(getStorefilesCount() + getCompactedFilesCount());
@@ -2110,7 +1727,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
       return getScanners(filesToReopen, cacheBlocks, false, false, matcher, startRow,
         includeStartRow, stopRow, includeStopRow, readPt, false);
     } finally {
-      this.lock.readLock().unlock();
+      this.storeEngine.readUnlock();
     }
   }
 
@@ -2176,41 +1793,20 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
   @Override
   public long getStorefilesSize() {
     // Include all StoreFiles
-    return getStorefilesSize(this.storeEngine.getStoreFileManager().getStorefiles(), sf -> true);
+    return StoreUtils.getStorefilesSize(this.storeEngine.getStoreFileManager().getStorefiles(),
+      sf -> true);
   }
 
   @Override
   public long getHFilesSize() {
     // Include only StoreFiles which are HFiles
-    return getStorefilesSize(this.storeEngine.getStoreFileManager().getStorefiles(),
+    return StoreUtils.getStorefilesSize(this.storeEngine.getStoreFileManager().getStorefiles(),
       HStoreFile::isHFile);
   }
 
-  private long getTotalUncompressedBytes(List<HStoreFile> files) {
-    return files.stream()
-      .mapToLong(file -> getStorefileFieldSize(file, StoreFileReader::getTotalUncompressedBytes))
-      .sum();
-  }
-
-  private long getStorefilesSize(Collection<HStoreFile> files, Predicate<HStoreFile> predicate) {
-    return files.stream().filter(predicate)
-      .mapToLong(file -> getStorefileFieldSize(file, StoreFileReader::length)).sum();
-  }
-
-  private long getStorefileFieldSize(HStoreFile file, ToLongFunction<StoreFileReader> f) {
-    if (file == null) {
-      return 0L;
-    }
-    StoreFileReader reader = file.getReader();
-    if (reader == null) {
-      return 0L;
-    }
-    return f.applyAsLong(reader);
-  }
-
   private long getStorefilesFieldSize(ToLongFunction<StoreFileReader> f) {
     return this.storeEngine.getStoreFileManager().getStorefiles().stream()
-      .mapToLong(file -> getStorefileFieldSize(file, f)).sum();
+      .mapToLong(file -> StoreUtils.getStorefileFieldSize(file, f)).sum();
   }
 
   @Override
@@ -2281,11 +1877,11 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
    */
   public void upsert(Iterable<Cell> cells, long readpoint, MemStoreSizing memstoreSizing)
       throws IOException {
-    this.lock.readLock().lock();
+    this.storeEngine.readLock();
     try {
       this.memstore.upsert(cells, readpoint, memstoreSizing);
     } finally {
-      this.lock.readLock().unlock();
+      this.storeEngine.readUnlock();
     }
   }
 
@@ -2338,7 +1934,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
         return false;
       }
       status.setStatus("Flushing " + this + ": reopening flushed file");
-      List<HStoreFile> storeFiles = commitStoreFiles(tempFiles, false);
+      List<HStoreFile> storeFiles = storeEngine.commitStoreFiles(tempFiles, false);
       for (HStoreFile sf : storeFiles) {
         StoreFileReader r = sf.getReader();
         if (LOG.isInfoEnabled()) {
@@ -2361,7 +1957,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
         }
       }
       // Add new file to store files. Clear snapshot too while we have the Store write lock.
-      return updateStorefiles(storeFiles, snapshot.getId());
+      return completeFlush(storeFiles, snapshot.getId());
     }
 
     @Override
@@ -2389,7 +1985,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
         // open the file as a store file (hfile link, etc)
         StoreFileInfo storeFileInfo =
           getRegionFileSystem().getStoreFileInfo(getColumnFamilyName(), file);
-        HStoreFile storeFile = createStoreFileAndReader(storeFileInfo);
+        HStoreFile storeFile = storeEngine.createStoreFileAndReader(storeFileInfo);
         storeFiles.add(storeFile);
         HStore.this.storeSize.addAndGet(storeFile.getReader().length());
         HStore.this.totalUncompressedBytes
@@ -2405,7 +2001,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
       if (dropMemstoreSnapshot && snapshot != null) {
         snapshotId = snapshot.getId();
       }
-      HStore.this.updateStorefiles(storeFiles, snapshotId);
+      HStore.this.completeFlush(storeFiles, snapshotId);
     }
 
     /**
@@ -2414,7 +2010,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
     @Override
     public void abort() throws IOException {
       if (snapshot != null) {
-        HStore.this.updateStorefiles(Collections.emptyList(), snapshot.getId());
+        HStore.this.completeFlush(Collections.emptyList(), snapshot.getId());
       }
     }
   }
@@ -2577,7 +2173,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
     // ensure other threads do not attempt to archive the same files on close()
     archiveLock.lock();
     try {
-      lock.readLock().lock();
+      storeEngine.readLock();
       Collection<HStoreFile> copyCompactedfiles = null;
       try {
         Collection<HStoreFile> compactedfiles =
@@ -2589,7 +2185,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
           LOG.trace("No compacted files to archive");
         }
       } finally {
-        lock.readLock().unlock();
+        storeEngine.readUnlock();
       }
       if (CollectionUtils.isNotEmpty(copyCompactedfiles)) {
         removeCompactedfiles(copyCompactedfiles, true);
@@ -2724,12 +2320,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
 
   private void clearCompactedfiles(List<HStoreFile> filesToRemove) throws IOException {
     LOG.trace("Clearing the compacted file {} from this store", filesToRemove);
-    try {
-      lock.writeLock().lock();
-      this.getStoreEngine().getStoreFileManager().removeCompactedFiles(filesToRemove);
-    } finally {
-      lock.writeLock().unlock();
-    }
+    storeEngine.removeCompactedFiles(filesToRemove);
   }
 
   @Override
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreContext.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreContext.java
index 2623350..2a9f968 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreContext.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreContext.java
@@ -23,6 +23,7 @@ import java.util.function.Supplier;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.io.HeapSize;
 import org.apache.hadoop.hbase.io.crypto.Encryption;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
@@ -108,6 +109,14 @@ public final class StoreContext implements HeapSize {
     return coprocessorHost;
   }
 
+  public RegionInfo getRegionInfo() {
+    return regionFileSystem.getRegionInfo();
+  }
+
+  public boolean isPrimaryReplicaStore() {
+    return getRegionInfo().getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID;
+  }
+
   public static Builder getBuilder() {
     return new Builder();
   }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreEngine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreEngine.java
index 60b3c3d..4033c33 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreEngine.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreEngine.java
@@ -19,38 +19,131 @@
 
 package org.apache.hadoop.hbase.regionserver;
 
+import com.google.errorprone.annotations.RestrictedApi;
 import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
-
+import java.util.Set;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.log.HBaseMarkers;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionPolicy;
 import org.apache.hadoop.hbase.regionserver.compactions.Compactor;
+import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTracker;
+import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory;
 import org.apache.hadoop.hbase.util.ReflectionUtils;
 import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
+import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
 
 /**
- * StoreEngine is a factory that can create the objects necessary for HStore to operate.
- * Since not all compaction policies, compactors and store file managers are compatible,
- * they are tied together and replaced together via StoreEngine-s.
+ * StoreEngine is a factory that can create the objects necessary for HStore to operate. Since not
+ * all compaction policies, compactors and store file managers are compatible, they are tied
+ * together and replaced together via StoreEngine-s.
+ * <p/>
+ * We expose read write lock methods to upper layer for store operations:<br/>
+ * <ul>
+ * <li>Locked in shared mode when the list of component stores is looked at:
+ * <ul>
+ * <li>all reads/writes to table data</li>
+ * <li>checking for split</li>
+ * </ul>
+ * </li>
+ * <li>Locked in exclusive mode when the list of component stores is modified:
+ * <ul>
+ * <li>closing</li>
+ * <li>completing a compaction</li>
+ * </ul>
+ * </li>
+ * </ul>
+ * <p/>
+ * It is a bit confusing that we have a StoreFileManager(SFM) and then a StoreFileTracker(SFT). As
+ * its name says, SFT is used to track the store files list. The reason why we have a SFT beside SFM
+ * is that, when introducing stripe compaction, we introduced the StoreEngine and also the SFM, but
+ * actually, the SFM here is not a general 'Manager', it is only designed to manage the in memory
+ * 'stripes', so we can select different store files when scanning or compacting. The 'tracking' of
+ * store files is actually done in {@link org.apache.hadoop.hbase.regionserver.HRegionFileSystem}
+ * and {@link HStore} before we have SFT. And since SFM is designed to only holds in memory states,
+ * we will hold write lock when updating it, the lock is also used to protect the normal read/write
+ * requests. This means we'd better not add IO operations to SFM. And also, no matter what the in
+ * memory state is, stripe or not, it does not effect how we track the store files. So consider all
+ * these facts, here we introduce a separated SFT to track the store files.
+ * <p/>
+ * Here, since we always need to update SFM and SFT almost at the same time, we introduce methods in
+ * StoreEngine directly to update them both, so upper layer just need to update StoreEngine once, to
+ * reduce the possible misuse.
  */
 @InterfaceAudience.Private
-public abstract class StoreEngine<SF extends StoreFlusher,
-    CP extends CompactionPolicy, C extends Compactor, SFM extends StoreFileManager> {
+public abstract class StoreEngine<SF extends StoreFlusher, CP extends CompactionPolicy,
+  C extends Compactor, SFM extends StoreFileManager> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(StoreEngine.class);
+
   protected SF storeFlusher;
   protected CP compactionPolicy;
   protected C compactor;
   protected SFM storeFileManager;
+  private Configuration conf;
+  private StoreContext ctx;
+  private RegionCoprocessorHost coprocessorHost;
+  private Function<String, ExecutorService> openStoreFileThreadPoolCreator;
+  private StoreFileTracker storeFileTracker;
+
+  private final ReadWriteLock storeLock = new ReentrantReadWriteLock();
 
   /**
-   * The name of the configuration parameter that specifies the class of
-   * a store engine that is used to manage and compact HBase store files.
+   * The name of the configuration parameter that specifies the class of a store engine that is used
+   * to manage and compact HBase store files.
    */
   public static final String STORE_ENGINE_CLASS_KEY = "hbase.hstore.engine.class";
 
-  private static final Class<? extends StoreEngine<?, ?, ?, ?>>
-    DEFAULT_STORE_ENGINE_CLASS = DefaultStoreEngine.class;
+  private static final Class<? extends StoreEngine<?, ?, ?, ?>> DEFAULT_STORE_ENGINE_CLASS =
+    DefaultStoreEngine.class;
+
+  /**
+   * Acquire read lock of this store.
+   */
+  public void readLock() {
+    storeLock.readLock().lock();
+  }
+
+  /**
+   * Release read lock of this store.
+   */
+  public void readUnlock() {
+    storeLock.readLock().unlock();
+  }
+
+  /**
+   * Acquire write lock of this store.
+   */
+  public void writeLock() {
+    storeLock.writeLock().lock();
+  }
+
+  /**
+   * Release write lock of this store.
+   */
+  public void writeUnlock() {
+    storeLock.writeLock().unlock();
+  }
 
   /**
    * @return Compaction policy to use.
@@ -80,6 +173,11 @@ public abstract class StoreEngine<SF extends StoreFlusher,
     return this.storeFlusher;
   }
 
+  private StoreFileTracker createStoreFileTracker(HStore store) {
+    return StoreFileTrackerFactory.create(store.conf, store.getRegionInfo().getTable(),
+      store.isPrimaryReplicaStore(), store.getStoreContext());
+  }
+
   /**
    * @param filesCompacting Files currently compacting
    * @return whether a compaction selection is possible
@@ -87,8 +185,8 @@ public abstract class StoreEngine<SF extends StoreFlusher,
   public abstract boolean needsCompaction(List<HStoreFile> filesCompacting);
 
   /**
-   * Creates an instance of a compaction context specific to this engine.
-   * Doesn't actually select or start a compaction. See CompactionContext class comment.
+   * Creates an instance of a compaction context specific to this engine. Doesn't actually select or
+   * start a compaction. See CompactionContext class comment.
    * @return New CompactionContext object.
    */
   public abstract CompactionContext createCompaction() throws IOException;
@@ -96,36 +194,347 @@ public abstract class StoreEngine<SF extends StoreFlusher,
   /**
    * Create the StoreEngine's components.
    */
-  protected abstract void createComponents(
-      Configuration conf, HStore store, CellComparator cellComparator) throws IOException;
+  protected abstract void createComponents(Configuration conf, HStore store,
+    CellComparator cellComparator) throws IOException;
 
-  private void createComponentsOnce(
-      Configuration conf, HStore store, CellComparator cellComparator) throws IOException {
-    assert compactor == null && compactionPolicy == null
-        && storeFileManager == null && storeFlusher == null;
+  protected final void createComponentsOnce(Configuration conf, HStore store,
+    CellComparator cellComparator) throws IOException {
+    assert compactor == null && compactionPolicy == null && storeFileManager == null &&
+      storeFlusher == null && storeFileTracker == null;
     createComponents(conf, store, cellComparator);
-    assert compactor != null && compactionPolicy != null
-        && storeFileManager != null && storeFlusher != null;
+    this.conf = conf;
+    this.ctx = store.getStoreContext();
+    this.coprocessorHost = store.getHRegion().getCoprocessorHost();
+    this.openStoreFileThreadPoolCreator = store.getHRegion()::getStoreFileOpenAndCloseThreadPool;
+    this.storeFileTracker = createStoreFileTracker(store);
+    assert compactor != null && compactionPolicy != null && storeFileManager != null &&
+      storeFlusher != null && storeFileTracker != null;
+  }
+
+  /**
+   * Create a writer for writing new store files.
+   * @return Writer for a new StoreFile
+   */
+  public StoreFileWriter createWriter(CreateStoreFileWriterParams params) throws IOException {
+    return storeFileTracker.createWriter(params);
+  }
+
+  public HStoreFile createStoreFileAndReader(Path p) throws IOException {
+    StoreFileInfo info = new StoreFileInfo(conf, ctx.getRegionFileSystem().getFileSystem(), p,
+      ctx.isPrimaryReplicaStore());
+    return createStoreFileAndReader(info);
+  }
+
+  public HStoreFile createStoreFileAndReader(StoreFileInfo info) throws IOException {
+    info.setRegionCoprocessorHost(coprocessorHost);
+    HStoreFile storeFile =
+      new HStoreFile(info, ctx.getFamily().getBloomFilterType(), ctx.getCacheConf());
+    storeFile.initReader();
+    return storeFile;
+  }
+
+  /**
+   * Validates a store file by opening and closing it. In HFileV2 this should not be an expensive
+   * operation.
+   * @param path the path to the store file
+   */
+  public void validateStoreFile(Path path) throws IOException {
+    HStoreFile storeFile = null;
+    try {
+      storeFile = createStoreFileAndReader(path);
+    } catch (IOException e) {
+      LOG.error("Failed to open store file : {}, keeping it in tmp location", path, e);
+      throw e;
+    } finally {
+      if (storeFile != null) {
+        storeFile.closeStoreFile(false);
+      }
+    }
+  }
+
+  private List<HStoreFile> openStoreFiles(Collection<StoreFileInfo> files, boolean warmup)
+    throws IOException {
+    if (CollectionUtils.isEmpty(files)) {
+      return Collections.emptyList();
+    }
+    // initialize the thread pool for opening store files in parallel..
+    ExecutorService storeFileOpenerThreadPool =
+      openStoreFileThreadPoolCreator.apply("StoreFileOpener-" +
+        ctx.getRegionInfo().getEncodedName() + "-" + ctx.getFamily().getNameAsString());
+    CompletionService<HStoreFile> completionService =
+      new ExecutorCompletionService<>(storeFileOpenerThreadPool);
+
+    int totalValidStoreFile = 0;
+    for (StoreFileInfo storeFileInfo : files) {
+      // The StoreFileInfo will carry store configuration down to HFile, we need to set it to
+      // our store's CompoundConfiguration here.
+      storeFileInfo.setConf(conf);
+      // open each store file in parallel
+      completionService.submit(() -> createStoreFileAndReader(storeFileInfo));
+      totalValidStoreFile++;
+    }
+
+    Set<String> compactedStoreFiles = new HashSet<>();
+    ArrayList<HStoreFile> results = new ArrayList<>(files.size());
+    IOException ioe = null;
+    try {
+      for (int i = 0; i < totalValidStoreFile; i++) {
+        try {
+          HStoreFile storeFile = completionService.take().get();
+          if (storeFile != null) {
+            LOG.debug("loaded {}", storeFile);
+            results.add(storeFile);
+            compactedStoreFiles.addAll(storeFile.getCompactedStoreFiles());
+          }
+        } catch (InterruptedException e) {
+          if (ioe == null) {
+            ioe = new InterruptedIOException(e.getMessage());
+          }
+        } catch (ExecutionException e) {
+          if (ioe == null) {
+            ioe = new IOException(e.getCause());
+          }
+        }
+      }
+    } finally {
+      storeFileOpenerThreadPool.shutdownNow();
+    }
+    if (ioe != null) {
+      // close StoreFile readers
+      boolean evictOnClose =
+        ctx.getCacheConf() != null ? ctx.getCacheConf().shouldEvictOnClose() : true;
+      for (HStoreFile file : results) {
+        try {
+          if (file != null) {
+            file.closeStoreFile(evictOnClose);
+          }
+        } catch (IOException e) {
+          LOG.warn("Could not close store file {}", file, e);
+        }
+      }
+      throw ioe;
+    }
+
+    // Should not archive the compacted store files when region warmup. See HBASE-22163.
+    if (!warmup) {
+      // Remove the compacted files from result
+      List<HStoreFile> filesToRemove = new ArrayList<>(compactedStoreFiles.size());
+      for (HStoreFile storeFile : results) {
+        if (compactedStoreFiles.contains(storeFile.getPath().getName())) {
+          LOG.warn("Clearing the compacted storefile {} from {}", storeFile, this);
+          storeFile.getReader().close(
+            storeFile.getCacheConf() != null ? storeFile.getCacheConf().shouldEvictOnClose() :
+              true);
+          filesToRemove.add(storeFile);
+        }
+      }
+      results.removeAll(filesToRemove);
+      if (!filesToRemove.isEmpty() && ctx.isPrimaryReplicaStore()) {
+        LOG.debug("Moving the files {} to archive", filesToRemove);
+        ctx.getRegionFileSystem().removeStoreFiles(ctx.getFamily().getNameAsString(),
+          filesToRemove);
+      }
+    }
+
+    return results;
+  }
+
+  public void initialize(boolean warmup) throws IOException {
+    List<StoreFileInfo> fileInfos = storeFileTracker.load();
+    List<HStoreFile> files = openStoreFiles(fileInfos, warmup);
+    storeFileManager.loadFiles(files);
+  }
+
+  public void refreshStoreFiles() throws IOException {
+    List<StoreFileInfo> fileInfos = storeFileTracker.load();
+    refreshStoreFilesInternal(fileInfos);
+  }
+
+  public void refreshStoreFiles(Collection<String> newFiles) throws IOException {
+    List<StoreFileInfo> storeFiles = new ArrayList<>(newFiles.size());
+    for (String file : newFiles) {
+      storeFiles
+        .add(ctx.getRegionFileSystem().getStoreFileInfo(ctx.getFamily().getNameAsString(), file));
+    }
+    refreshStoreFilesInternal(storeFiles);
+  }
+
+  /**
+   * Checks the underlying store files, and opens the files that have not been opened, and removes
+   * the store file readers for store files no longer available. Mainly used by secondary region
+   * replicas to keep up to date with the primary region files.
+   */
+  private void refreshStoreFilesInternal(Collection<StoreFileInfo> newFiles) throws IOException {
+    Collection<HStoreFile> currentFiles = storeFileManager.getStorefiles();
+    Collection<HStoreFile> compactedFiles = storeFileManager.getCompactedfiles();
+    if (currentFiles == null) {
+      currentFiles = Collections.emptySet();
+    }
+    if (newFiles == null) {
+      newFiles = Collections.emptySet();
+    }
+    if (compactedFiles == null) {
+      compactedFiles = Collections.emptySet();
+    }
+
+    HashMap<StoreFileInfo, HStoreFile> currentFilesSet = new HashMap<>(currentFiles.size());
+    for (HStoreFile sf : currentFiles) {
+      currentFilesSet.put(sf.getFileInfo(), sf);
+    }
+    HashMap<StoreFileInfo, HStoreFile> compactedFilesSet = new HashMap<>(compactedFiles.size());
+    for (HStoreFile sf : compactedFiles) {
+      compactedFilesSet.put(sf.getFileInfo(), sf);
+    }
+
+    Set<StoreFileInfo> newFilesSet = new HashSet<StoreFileInfo>(newFiles);
+    // Exclude the files that have already been compacted
+    newFilesSet = Sets.difference(newFilesSet, compactedFilesSet.keySet());
+    Set<StoreFileInfo> toBeAddedFiles = Sets.difference(newFilesSet, currentFilesSet.keySet());
+    Set<StoreFileInfo> toBeRemovedFiles = Sets.difference(currentFilesSet.keySet(), newFilesSet);
+
+    if (toBeAddedFiles.isEmpty() && toBeRemovedFiles.isEmpty()) {
+      return;
+    }
+
+    LOG.info("Refreshing store files for " + this + " files to add: " + toBeAddedFiles +
+      " files to remove: " + toBeRemovedFiles);
+
+    Set<HStoreFile> toBeRemovedStoreFiles = new HashSet<>(toBeRemovedFiles.size());
+    for (StoreFileInfo sfi : toBeRemovedFiles) {
+      toBeRemovedStoreFiles.add(currentFilesSet.get(sfi));
+    }
+
+    // try to open the files
+    List<HStoreFile> openedFiles = openStoreFiles(toBeAddedFiles, false);
+
+    // propogate the file changes to the underlying store file manager
+    replaceStoreFiles(toBeRemovedStoreFiles, openedFiles); // won't throw an exception
+  }
+
+  /**
+   * Commit the given {@code files}.
+   * <p/>
+   * We will move the file into data directory, and open it.
+   * @param files the files want to commit
+   * @param validate whether to validate the store files
+   * @return the committed store files
+   */
+  public List<HStoreFile> commitStoreFiles(List<Path> files, boolean validate) throws IOException {
+    List<HStoreFile> committedFiles = new ArrayList<>(files.size());
+    HRegionFileSystem hfs = ctx.getRegionFileSystem();
+    String familyName = ctx.getFamily().getNameAsString();
+    Path storeDir = hfs.getStoreDir(familyName);
+    for (Path file : files) {
+      try {
+        if (validate) {
+          validateStoreFile(file);
+        }
+        Path committedPath;
+        // As we want to support writing to data directory directly, here we need to check whether
+        // the store file is already in the right place
+        if (file.getParent() != null && file.getParent().equals(storeDir)) {
+          // already in the right place, skip renmaing
+          committedPath = file;
+        } else {
+          // Write-out finished successfully, move into the right spot
+          committedPath = hfs.commitStoreFile(familyName, file);
+        }
+        HStoreFile sf = createStoreFileAndReader(committedPath);
+        committedFiles.add(sf);
+      } catch (IOException e) {
+        LOG.error("Failed to commit store file {}", file, e);
+        // Try to delete the files we have committed before.
+        // It is OK to fail when deleting as leaving the file there does not cause any data
+        // corruption problem. It just introduces some duplicated data which may impact read
+        // performance a little when reading before compaction.
+        for (HStoreFile sf : committedFiles) {
+          Path pathToDelete = sf.getPath();
+          try {
+            sf.deleteStoreFile();
+          } catch (IOException deleteEx) {
+            LOG.warn(HBaseMarkers.FATAL, "Failed to delete committed store file {}", pathToDelete,
+              deleteEx);
+          }
+        }
+        throw new IOException("Failed to commit the flush", e);
+      }
+    }
+    return committedFiles;
+  }
+
+  @FunctionalInterface
+  public interface IOExceptionRunnable {
+    void run() throws IOException;
+  }
+
+  /**
+   * Add the store files to store file manager, and also record it in the store file tracker.
+   * <p/>
+   * The {@code actionAfterAdding} will be executed after the insertion to store file manager, under
+   * the lock protection. Usually this is for clear the memstore snapshot.
+   */
+  public void addStoreFiles(Collection<HStoreFile> storeFiles,
+    IOExceptionRunnable actionAfterAdding) throws IOException {
+    storeFileTracker.add(StoreUtils.toStoreFileInfo(storeFiles));
+    writeLock();
+    try {
+      storeFileManager.insertNewFiles(storeFiles);
+      actionAfterAdding.run();
+    } finally {
+      // We need the lock, as long as we are updating the storeFiles
+      // or changing the memstore. Let us release it before calling
+      // notifyChangeReadersObservers. See HBASE-4485 for a possible
+      // deadlock scenario that could have happened if continue to hold
+      // the lock.
+      writeUnlock();
+    }
+  }
+
+  public void replaceStoreFiles(Collection<HStoreFile> compactedFiles,
+    Collection<HStoreFile> newFiles) throws IOException {
+    storeFileTracker.replace(StoreUtils.toStoreFileInfo(compactedFiles),
+      StoreUtils.toStoreFileInfo(newFiles));
+    writeLock();
+    try {
+      storeFileManager.addCompactionResults(compactedFiles, newFiles);
+    } finally {
+      writeUnlock();
+    }
+  }
+
+  public void removeCompactedFiles(Collection<HStoreFile> compactedFiles) {
+    writeLock();
+    try {
+      storeFileManager.removeCompactedFiles(compactedFiles);
+    } finally {
+      writeUnlock();
+    }
   }
 
   /**
    * Create the StoreEngine configured for the given Store.
-   * @param store The store. An unfortunate dependency needed due to it
-   *              being passed to coprocessors via the compactor.
+   * @param store The store. An unfortunate dependency needed due to it being passed to coprocessors
+   *          via the compactor.
    * @param conf Store configuration.
    * @param cellComparator CellComparator for storeFileManager.
    * @return StoreEngine to use.
    */
-  public static StoreEngine<?, ?, ?, ?> create(
-      HStore store, Configuration conf, CellComparator cellComparator) throws IOException {
+  public static StoreEngine<?, ?, ?, ?> create(HStore store, Configuration conf,
+    CellComparator cellComparator) throws IOException {
     String className = conf.get(STORE_ENGINE_CLASS_KEY, DEFAULT_STORE_ENGINE_CLASS.getName());
     try {
-      StoreEngine<?,?,?,?> se = ReflectionUtils.instantiateWithCustomCtor(
-          className, new Class[] { }, new Object[] { });
+      StoreEngine<?, ?, ?, ?> se =
+        ReflectionUtils.instantiateWithCustomCtor(className, new Class[] {}, new Object[] {});
       se.createComponentsOnce(conf, store, cellComparator);
       return se;
     } catch (Exception e) {
       throw new IOException("Unable to load configured store engine '" + className + "'", e);
     }
   }
+
+  @RestrictedApi(explanation = "Should only be called in TestHStore", link = "",
+    allowedOnPath = ".*/TestHStore.java")
+  ReadWriteLock getLock() {
+    return storeLock;
+  }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileManager.java
index 27127f3..a40b209 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileManager.java
@@ -18,6 +18,7 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
+import com.google.errorprone.annotations.RestrictedApi;
 import java.io.IOException;
 import java.util.Collection;
 import java.util.Comparator;
@@ -49,12 +50,16 @@ public interface StoreFileManager {
    * Loads the initial store files into empty StoreFileManager.
    * @param storeFiles The files to load.
    */
+  @RestrictedApi(explanation = "Should only be called in StoreEngine", link = "",
+    allowedOnPath = ".*(/org/apache/hadoop/hbase/regionserver/StoreEngine.java|/src/test/.*)")
   void loadFiles(List<HStoreFile> storeFiles);
 
   /**
    * Adds new files, either for from MemStore flush or bulk insert, into the structure.
    * @param sfs New store files.
    */
+  @RestrictedApi(explanation = "Should only be called in StoreEngine", link = "",
+    allowedOnPath = ".*(/org/apache/hadoop/hbase/regionserver/StoreEngine.java|/src/test/.*)")
   void insertNewFiles(Collection<HStoreFile> sfs);
 
   /**
@@ -62,12 +67,16 @@ public interface StoreFileManager {
    * @param compactedFiles The input files for the compaction.
    * @param results The resulting files for the compaction.
    */
+  @RestrictedApi(explanation = "Should only be called in StoreEngine", link = "",
+    allowedOnPath = ".*(/org/apache/hadoop/hbase/regionserver/StoreEngine.java|/src/test/.*)")
   void addCompactionResults(Collection<HStoreFile> compactedFiles, Collection<HStoreFile> results);
 
   /**
    * Remove the compacted files
    * @param compactedFiles the list of compacted files
    */
+  @RestrictedApi(explanation = "Should only be called in StoreEngine", link = "",
+    allowedOnPath = ".*(/org/apache/hadoop/hbase/regionserver/StoreEngine.java|/src/test/.*)")
   void removeCompactedFiles(Collection<HStoreFile> compactedFiles);
 
   /**
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFlusher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFlusher.java
index 67eb375..1095854 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFlusher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFlusher.java
@@ -69,10 +69,17 @@ abstract class StoreFlusher {
     writer.close();
   }
 
+  protected final StoreFileWriter createWriter(MemStoreSnapshot snapshot, boolean alwaysIncludesTag)
+    throws IOException {
+    return store.getStoreEngine()
+      .createWriter(CreateStoreFileWriterParams.create().maxKeyCount(snapshot.getCellsCount())
+        .compression(store.getColumnFamilyDescriptor().getCompressionType()).isCompaction(false)
+        .includeMVCCReadpoint(true).includesTag(alwaysIncludesTag || snapshot.isTagsPresent())
+        .shouldDropBehind(false));
+  }
 
   /**
    * Creates the scanner for flushing snapshot. Also calls coprocessors.
-   * @param snapshotScanners
    * @return The scanner; null if coprocessor is canceling the flush.
    */
   protected final InternalScanner createScanner(List<KeyValueScanner> snapshotScanners,
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreUtils.java
index 454b244..10a9330 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreUtils.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreUtils.java
@@ -20,10 +20,13 @@ package org.apache.hadoop.hbase.regionserver;
 
 import java.io.IOException;
 import java.util.Collection;
+import java.util.List;
 import java.util.Optional;
 import java.util.OptionalInt;
 import java.util.OptionalLong;
-
+import java.util.function.Predicate;
+import java.util.function.ToLongFunction;
+import java.util.stream.Collectors;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
@@ -42,10 +45,13 @@ import org.slf4j.LoggerFactory;
  * Utility functions for region server storage layer.
  */
 @InterfaceAudience.Private
-public class StoreUtils {
+public final class StoreUtils {
 
   private static final Logger LOG = LoggerFactory.getLogger(StoreUtils.class);
 
+  private StoreUtils() {
+  }
+
   /**
    * Creates a deterministic hash code for store file collection.
    */
@@ -171,4 +177,31 @@ public class StoreUtils {
     return new CompoundConfiguration().add(conf).addBytesMap(td.getValues())
         .addStringMap(cfd.getConfiguration()).addBytesMap(cfd.getValues());
   }
+
+  public static List<StoreFileInfo> toStoreFileInfo(Collection<HStoreFile> storefiles) {
+    return storefiles.stream().map(HStoreFile::getFileInfo).collect(Collectors.toList());
+  }
+
+  public static long getTotalUncompressedBytes(List<HStoreFile> files) {
+    return files.stream()
+      .mapToLong(file -> getStorefileFieldSize(file, StoreFileReader::getTotalUncompressedBytes))
+      .sum();
+  }
+
+  public static long getStorefilesSize(Collection<HStoreFile> files,
+    Predicate<HStoreFile> predicate) {
+    return files.stream().filter(predicate)
+      .mapToLong(file -> getStorefileFieldSize(file, StoreFileReader::length)).sum();
+  }
+
+  public static long getStorefileFieldSize(HStoreFile file, ToLongFunction<StoreFileReader> f) {
+    if (file == null) {
+      return 0L;
+    }
+    StoreFileReader reader = file.getReader();
+    if (reader == null) {
+      return 0L;
+    }
+    return f.applyAsLong(reader);
+  }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreEngine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreEngine.java
index 14863a6..bfb3f64 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreEngine.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreEngine.java
@@ -20,20 +20,19 @@ package org.apache.hadoop.hbase.regionserver;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequestImpl;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequestImpl;
 import org.apache.hadoop.hbase.regionserver.compactions.StripeCompactionPolicy;
 import org.apache.hadoop.hbase.regionserver.compactions.StripeCompactor;
 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
 import org.apache.hadoop.hbase.security.User;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFlusher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFlusher.java
index 1560aef..f8183b7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFlusher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFlusher.java
@@ -70,7 +70,7 @@ public class StripeStoreFlusher extends StoreFlusher {
     StripeMultiFileWriter mw = null;
     try {
       mw = req.createWriter(); // Writer according to the policy.
-      StripeMultiFileWriter.WriterFactory factory = createWriterFactory(cellsCount);
+      StripeMultiFileWriter.WriterFactory factory = createWriterFactory(snapshot);
       StoreScanner storeScanner = (scanner instanceof StoreScanner) ? (StoreScanner)scanner : null;
       mw.init(storeScanner, factory);
 
@@ -98,13 +98,12 @@ public class StripeStoreFlusher extends StoreFlusher {
     return result;
   }
 
-  private StripeMultiFileWriter.WriterFactory createWriterFactory(final long kvCount) {
+  private StripeMultiFileWriter.WriterFactory createWriterFactory(MemStoreSnapshot snapshot) {
     return new StripeMultiFileWriter.WriterFactory() {
       @Override
       public StoreFileWriter createWriter() throws IOException {
-        StoreFileWriter writer = store.createWriterInTmp(kvCount,
-            store.getColumnFamilyDescriptor().getCompressionType(), false, true, true, false);
-        return writer;
+        // XXX: it used to always pass true for includesTag, re-consider?
+        return StripeStoreFlusher.this.createWriter(snapshot, true);
       }
     };
   }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/AbstractMultiOutputCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/AbstractMultiOutputCompactor.java
index 42841bf..533be17 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/AbstractMultiOutputCompactor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/AbstractMultiOutputCompactor.java
@@ -51,13 +51,14 @@ public abstract class AbstractMultiOutputCompactor<T extends AbstractMultiFileWr
     WriterFactory writerFactory = new WriterFactory() {
       @Override
       public StoreFileWriter createWriter() throws IOException {
-        return createTmpWriter(fd, shouldDropBehind, major);
+        return AbstractMultiOutputCompactor.this.createWriter(fd, shouldDropBehind, major);
       }
 
       @Override
       public StoreFileWriter createWriterWithStoragePolicy(String fileStoragePolicy)
-          throws IOException {
-        return createTmpWriter(fd, shouldDropBehind, fileStoragePolicy, major);
+        throws IOException {
+        return AbstractMultiOutputCompactor.this.createWriter(fd, shouldDropBehind,
+          fileStoragePolicy, major);
       }
     };
     // Prepare multi-writer, and perform the compaction using scanner and writer.
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
index 7f70e02..8178fb1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
@@ -28,7 +28,6 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 import java.util.Map;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
@@ -38,6 +37,7 @@ import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.io.hfile.HFileInfo;
 import org.apache.hadoop.hbase.regionserver.CellSink;
+import org.apache.hadoop.hbase.regionserver.CreateStoreFileWriterParams;
 import org.apache.hadoop.hbase.regionserver.HStore;
 import org.apache.hadoop.hbase.regionserver.HStoreFile;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
@@ -60,6 +60,7 @@ import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.com.google.common.io.Closeables;
 
 /**
@@ -260,29 +261,32 @@ public abstract class Compactor<T extends CellSink> {
     }
   };
 
+  protected final CreateStoreFileWriterParams createParams(FileDetails fd, boolean shouldDropBehind,
+    boolean major) {
+    return CreateStoreFileWriterParams.create().maxKeyCount(fd.maxKeyCount)
+      .compression(major ? majorCompactionCompression : minorCompactionCompression)
+      .isCompaction(true).includeMVCCReadpoint(fd.maxMVCCReadpoint > 0)
+      .includesTag(fd.maxTagsLength > 0).shouldDropBehind(shouldDropBehind)
+      .totalCompactedFilesSize(fd.totalCompactedFilesSize);
+  }
+
   /**
-   * Creates a writer for a new file in a temporary directory.
+   * Creates a writer for a new file.
    * @param fd The file details.
-   * @return Writer for a new StoreFile in the tmp dir.
+   * @return Writer for a new StoreFile
    * @throws IOException if creation failed
    */
-  protected final StoreFileWriter createTmpWriter(FileDetails fd, boolean shouldDropBehind, boolean major)
-      throws IOException {
+  protected final StoreFileWriter createWriter(FileDetails fd, boolean shouldDropBehind,
+    boolean major) throws IOException {
     // When all MVCC readpoints are 0, don't write them.
     // See HBASE-8166, HBASE-12600, and HBASE-13389.
-    return store.createWriterInTmp(fd.maxKeyCount,
-        major ? majorCompactionCompression : minorCompactionCompression,
-        true, fd.maxMVCCReadpoint > 0,
-        fd.maxTagsLength > 0, shouldDropBehind, fd.totalCompactedFilesSize,
-        HConstants.EMPTY_STRING);
+    return store.getStoreEngine().createWriter(createParams(fd, shouldDropBehind, major));
   }
 
-  protected final StoreFileWriter createTmpWriter(FileDetails fd, boolean shouldDropBehind,
-      String fileStoragePolicy, boolean major) throws IOException {
-    return store.createWriterInTmp(fd.maxKeyCount,
-      major ? majorCompactionCompression : minorCompactionCompression,
-      true, fd.maxMVCCReadpoint > 0,
-      fd.maxTagsLength > 0, shouldDropBehind, fd.totalCompactedFilesSize, fileStoragePolicy);
+  protected final StoreFileWriter createWriter(FileDetails fd, boolean shouldDropBehind,
+    String fileStoragePolicy, boolean major) throws IOException {
+    return store.getStoreEngine()
+      .createWriter(createParams(fd, shouldDropBehind, major).fileStoragePolicy(fileStoragePolicy));
   }
 
   private ScanInfo preCompactScannerOpen(CompactionRequestImpl request, ScanType scanType,
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java
index 49d3e8e..afa2429 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java
@@ -45,14 +45,14 @@ public class DefaultCompactor extends Compactor<StoreFileWriter> {
   }
 
   private final CellSinkFactory<StoreFileWriter> writerFactory =
-      new CellSinkFactory<StoreFileWriter>() {
-        @Override
-        public StoreFileWriter createWriter(InternalScanner scanner,
-            org.apache.hadoop.hbase.regionserver.compactions.Compactor.FileDetails fd,
-            boolean shouldDropBehind, boolean major) throws IOException {
-          return createTmpWriter(fd, shouldDropBehind, major);
-        }
-      };
+    new CellSinkFactory<StoreFileWriter>() {
+      @Override
+      public StoreFileWriter createWriter(InternalScanner scanner,
+        org.apache.hadoop.hbase.regionserver.compactions.Compactor.FileDetails fd,
+        boolean shouldDropBehind, boolean major) throws IOException {
+        return DefaultCompactor.this.createWriter(fd, shouldDropBehind, major);
+      }
+    };
 
   /**
    * Do a minor/major compaction on an explicit set of storefiles from a Store.
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/DefaultStoreFileTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/DefaultStoreFileTracker.java
new file mode 100644
index 0000000..d4c9a86
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/DefaultStoreFileTracker.java
@@ -0,0 +1,61 @@
+/**
+ * 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.regionserver.storefiletracker;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.regionserver.StoreContext;
+import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * The default implementation for store file tracker, where we do not persist the store file list,
+ * and use listing when loading store files.
+ */
+@InterfaceAudience.Private
+class DefaultStoreFileTracker extends StoreFileTrackerBase {
+
+  public DefaultStoreFileTracker(Configuration conf, TableName tableName, boolean isPrimaryReplica,
+    StoreContext ctx) {
+    super(conf, tableName, isPrimaryReplica, ctx);
+  }
+
+  @Override
+  public List<StoreFileInfo> load() throws IOException {
+    return ctx.getRegionFileSystem().getStoreFiles(ctx.getFamily().getNameAsString());
+  }
+
+  @Override
+  public boolean requireWritingToTmpDirFirst() {
+    return true;
+  }
+
+  @Override
+  protected void doAddNewStoreFiles(Collection<StoreFileInfo> newFiles) throws IOException {
+    // NOOP
+  }
+
+  @Override
+  protected void doAddCompactionResults(Collection<StoreFileInfo> compactedFiles,
+    Collection<StoreFileInfo> newFiles) throws IOException {
+    // NOOP
+  }
+}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java
new file mode 100644
index 0000000..aadedc8
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java
@@ -0,0 +1,75 @@
+/**
+ * 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.regionserver.storefiletracker;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.List;
+import org.apache.hadoop.hbase.regionserver.CreateStoreFileWriterParams;
+import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
+import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * An interface to define how we track the store files for a give store.
+ * <p/>
+ * In the old time, we will write store to a tmp directory first, and then rename it to the actual
+ * data file. And once a store file is under data directory, we will consider it as 'committed'. And
+ * we need to do listing when loading store files.
+ * <p/>
+ * When cloud age is coming, now we want to store the store files on object storage, where rename
+ * and list are not as cheap as on HDFS, especially rename. Although introducing a metadata
+ * management layer for object storage could solve the problem, but we still want HBase to run on
+ * pure object storage, so here we introduce this interface to abstract how we track the store
+ * files. For the old implementation, we just persist nothing here, and do listing to load store
+ * files. When running on object storage, we could persist the store file list in a system region,
+ * or in a file on the object storage, to make it possible to write directly into the data directory
+ * to avoid renaming, and also avoid listing when loading store files.
+ * <p/>
+ * The implementation requires to be thread safe as flush and compaction may occur as the same time,
+ * and we could also do multiple compactions at the same time. As the implementation may choose to
+ * persist the store file list to external storage, which could be slow, it is the duty for the
+ * callers to not call it inside a lock which may block normal read/write requests.
+ */
+@InterfaceAudience.Private
+public interface StoreFileTracker {
+
+  /**
+   * Load the store files list when opening a region.
+   */
+  List<StoreFileInfo> load() throws IOException;
+
+  /**
+   * Add new store files.
+   * <p/>
+   * Used for flush and bulk load.
+   */
+  void add(Collection<StoreFileInfo> newFiles) throws IOException;
+
+  /**
+   * Add new store files and remove compacted store files after compaction.
+   */
+  void replace(Collection<StoreFileInfo> compactedFiles, Collection<StoreFileInfo> newFiles)
+    throws IOException;
+
+  /**
+   * Create a writer for writing new store files.
+   * @return Writer for a new StoreFile
+   */
+  StoreFileWriter createWriter(CreateStoreFileWriterParams params) throws IOException;
+}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java
new file mode 100644
index 0000000..2451f45
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java
@@ -0,0 +1,178 @@
+/**
+ * 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.regionserver.storefiletracker;
+
+import java.io.IOException;
+import java.util.Collection;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.io.compress.Compression;
+import org.apache.hadoop.hbase.io.crypto.Encryption;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.io.hfile.HFile;
+import org.apache.hadoop.hbase.io.hfile.HFileContext;
+import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
+import org.apache.hadoop.hbase.regionserver.CreateStoreFileWriterParams;
+import org.apache.hadoop.hbase.regionserver.StoreContext;
+import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
+import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
+import org.apache.hadoop.hbase.regionserver.StoreUtils;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Base class for all store file tracker.
+ * <p/>
+ * Mainly used to place the common logic to skip persistent for secondary replicas.
+ */
+@InterfaceAudience.Private
+abstract class StoreFileTrackerBase implements StoreFileTracker {
+
+  private static final Logger LOG = LoggerFactory.getLogger(StoreFileTrackerBase.class);
+
+  protected final Configuration conf;
+
+  protected final TableName tableName;
+
+  protected final boolean isPrimaryReplica;
+
+  protected final StoreContext ctx;
+
+  private volatile boolean cacheOnWriteLogged;
+
+  protected StoreFileTrackerBase(Configuration conf, TableName tableName, boolean isPrimaryReplica,
+    StoreContext ctx) {
+    this.conf = conf;
+    this.tableName = tableName;
+    this.isPrimaryReplica = isPrimaryReplica;
+    this.ctx = ctx;
+  }
+
+  @Override
+  public final void add(Collection<StoreFileInfo> newFiles) throws IOException {
+    if (isPrimaryReplica) {
+      doAddNewStoreFiles(newFiles);
+    }
+  }
+
+  @Override
+  public final void replace(Collection<StoreFileInfo> compactedFiles,
+    Collection<StoreFileInfo> newFiles) throws IOException {
+    if (isPrimaryReplica) {
+      doAddCompactionResults(compactedFiles, newFiles);
+    }
+  }
+
+  private HFileContext createFileContext(Compression.Algorithm compression,
+    boolean includeMVCCReadpoint, boolean includesTag, Encryption.Context encryptionContext) {
+    if (compression == null) {
+      compression = HFile.DEFAULT_COMPRESSION_ALGORITHM;
+    }
+    ColumnFamilyDescriptor family = ctx.getFamily();
+    HFileContext hFileContext = new HFileContextBuilder().withIncludesMvcc(includeMVCCReadpoint)
+      .withIncludesTags(includesTag).withCompression(compression)
+      .withCompressTags(family.isCompressTags()).withChecksumType(StoreUtils.getChecksumType(conf))
+      .withBytesPerCheckSum(StoreUtils.getBytesPerChecksum(conf))
+      .withBlockSize(family.getBlocksize()).withHBaseCheckSum(true)
+      .withDataBlockEncoding(family.getDataBlockEncoding()).withEncryptionContext(encryptionContext)
+      .withCreateTime(EnvironmentEdgeManager.currentTime()).withColumnFamily(family.getName())
+      .withTableName(tableName.getName()).withCellComparator(ctx.getComparator()).build();
+    return hFileContext;
+  }
+
+  @Override
+  public final StoreFileWriter createWriter(CreateStoreFileWriterParams params)
+    throws IOException {
+    if (!isPrimaryReplica) {
+      throw new IllegalStateException("Should not call create writer on secondary replicas");
+    }
+    // creating new cache config for each new writer
+    final CacheConfig cacheConf = ctx.getCacheConf();
+    final CacheConfig writerCacheConf = new CacheConfig(cacheConf);
+    long totalCompactedFilesSize = params.totalCompactedFilesSize();
+    if (params.isCompaction()) {
+      // Don't cache data on write on compactions, unless specifically configured to do so
+      // Cache only when total file size remains lower than configured threshold
+      final boolean cacheCompactedBlocksOnWrite = cacheConf.shouldCacheCompactedBlocksOnWrite();
+      // if data blocks are to be cached on write
+      // during compaction, we should forcefully
+      // cache index and bloom blocks as well
+      if (cacheCompactedBlocksOnWrite &&
+        totalCompactedFilesSize <= cacheConf.getCacheCompactedBlocksOnWriteThreshold()) {
+        writerCacheConf.enableCacheOnWrite();
+        if (!cacheOnWriteLogged) {
+          LOG.info("For {} , cacheCompactedBlocksOnWrite is true, hence enabled " +
+            "cacheOnWrite for Data blocks, Index blocks and Bloom filter blocks", this);
+          cacheOnWriteLogged = true;
+        }
+      } else {
+        writerCacheConf.setCacheDataOnWrite(false);
+        if (totalCompactedFilesSize > cacheConf.getCacheCompactedBlocksOnWriteThreshold()) {
+          // checking condition once again for logging
+          LOG.debug(
+            "For {}, setting cacheCompactedBlocksOnWrite as false as total size of compacted " +
+              "files - {}, is greater than cacheCompactedBlocksOnWriteThreshold - {}",
+            this, totalCompactedFilesSize, cacheConf.getCacheCompactedBlocksOnWriteThreshold());
+        }
+      }
+    } else {
+      final boolean shouldCacheDataOnWrite = cacheConf.shouldCacheDataOnWrite();
+      if (shouldCacheDataOnWrite) {
+        writerCacheConf.enableCacheOnWrite();
+        if (!cacheOnWriteLogged) {
+          LOG.info("For {} , cacheDataOnWrite is true, hence enabled cacheOnWrite for " +
+            "Index blocks and Bloom filter blocks", this);
+          cacheOnWriteLogged = true;
+        }
+      }
+    }
+    Encryption.Context encryptionContext = ctx.getEncryptionContext();
+    HFileContext hFileContext = createFileContext(params.compression(),
+      params.includeMVCCReadpoint(), params.includesTag(), encryptionContext);
+    Path outputDir;
+    if (requireWritingToTmpDirFirst()) {
+      outputDir =
+        new Path(ctx.getRegionFileSystem().getTempDir(), ctx.getFamily().getNameAsString());
+    } else {
+      throw new UnsupportedOperationException("not supported yet");
+    }
+    StoreFileWriter.Builder builder =
+      new StoreFileWriter.Builder(conf, writerCacheConf, ctx.getRegionFileSystem().getFileSystem())
+        .withOutputDir(outputDir).withBloomType(ctx.getBloomFilterType())
+        .withMaxKeyCount(params.maxKeyCount()).withFavoredNodes(ctx.getFavoredNodes())
+        .withFileContext(hFileContext).withShouldDropCacheBehind(params.shouldDropBehind())
+        .withCompactedFilesSupplier(ctx.getCompactedFilesSupplier())
+        .withFileStoragePolicy(params.fileStoragePolicy());
+    return builder.build();
+  }
+
+  /**
+   * Whether the implementation of this tracker requires you to write to temp directory first, i.e,
+   * does not allow broken store files under the actual data directory.
+   */
+  protected abstract boolean requireWritingToTmpDirFirst();
+
+  protected abstract void doAddNewStoreFiles(Collection<StoreFileInfo> newFiles) throws IOException;
+
+  protected abstract void doAddCompactionResults(Collection<StoreFileInfo> compactedFiles,
+    Collection<StoreFileInfo> newFiles) throws IOException;
+}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java
new file mode 100644
index 0000000..4f7231b
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java
@@ -0,0 +1,35 @@
+/**
+ * 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.regionserver.storefiletracker;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.regionserver.StoreContext;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Factory method for creating store file tracker.
+ */
+@InterfaceAudience.Private
+public final class StoreFileTrackerFactory {
+
+  public static StoreFileTracker create(Configuration conf, TableName tableName,
+    boolean isPrimaryReplica, StoreContext ctx) {
+    return new DefaultStoreFileTracker(conf, tableName, isPrimaryReplica, ctx);
+  }
+}
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 f765b35..2112b97 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
@@ -101,7 +101,6 @@ class MajorCompactionRequest {
 
   boolean shouldCFBeCompacted(HRegionFileSystem fileSystem, String family, long ts)
       throws IOException {
-
     // do we have any store files?
     Collection<StoreFileInfo> storeFiles = fileSystem.getStoreFiles(family);
     if (storeFiles == null) {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java
index 8adffd3..c869e5e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java
@@ -210,11 +210,13 @@ public class TestIOFencing {
 
     @Override
     protected void refreshStoreSizeAndTotalBytes() throws IOException {
-      try {
-        r.compactionsWaiting.countDown();
-        r.compactionsBlocked.await();
-      } catch (InterruptedException ex) {
-        throw new IOException(ex);
+      if (r != null) {
+        try {
+          r.compactionsWaiting.countDown();
+          r.compactionsBlocked.await();
+        } catch (InterruptedException ex) {
+          throw new IOException(ex);
+        }
       }
       super.refreshStoreSizeAndTotalBytes();
     }
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 7d6d624..f825905 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
@@ -215,8 +215,10 @@ public class TestCacheOnWriteInSchema {
   @Test
   public void testCacheOnWriteInSchema() throws IOException {
     // Write some random data into the store
-    StoreFileWriter writer = store.createWriterInTmp(Integer.MAX_VALUE,
-        HFile.DEFAULT_COMPRESSION_ALGORITHM, false, true, false, false);
+    StoreFileWriter writer = store.getStoreEngine()
+      .createWriter(CreateStoreFileWriterParams.create().maxKeyCount(Integer.MAX_VALUE)
+        .compression(HFile.DEFAULT_COMPRESSION_ALGORITHM).isCompaction(false)
+        .includeMVCCReadpoint(true).includesTag(false).shouldDropBehind(false));
     writeStoreFile(writer);
     writer.close();
     // Verify the block types of interest were cached on write
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultStoreEngine.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultStoreEngine.java
index d36eb11..3784876 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultStoreEngine.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultStoreEngine.java
@@ -21,7 +21,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.CellComparatorImpl;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactor;
 import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
@@ -65,9 +65,12 @@ public class TestDefaultStoreEngine {
         DummyCompactionPolicy.class.getName());
     conf.set(DefaultStoreEngine.DEFAULT_STORE_FLUSHER_CLASS_KEY,
         DummyStoreFlusher.class.getName());
+    HRegion mockRegion = Mockito.mock(HRegion.class);
     HStore mockStore = Mockito.mock(HStore.class);
-    Mockito.when(mockStore.getRegionInfo()).thenReturn(HRegionInfo.FIRST_META_REGIONINFO);
-    StoreEngine<?, ?, ?, ?> se = StoreEngine.create(mockStore, conf, CellComparatorImpl.COMPARATOR);
+    Mockito.when(mockStore.getRegionInfo()).thenReturn(RegionInfoBuilder.FIRST_META_REGIONINFO);
+    Mockito.when(mockStore.getHRegion()).thenReturn(mockRegion);
+    StoreEngine<?, ?, ?, ?> se =
+      StoreEngine.create(mockStore, conf, CellComparatorImpl.COMPARATOR);
     Assert.assertTrue(se instanceof DefaultStoreEngine);
     Assert.assertTrue(se.getCompactionPolicy() instanceof DummyCompactionPolicy);
     Assert.assertTrue(se.getStoreFlusher() instanceof DummyStoreFlusher);
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 839cf34..0a1a8e8 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
@@ -5746,7 +5746,7 @@ public class TestHRegion {
       Collection<HStoreFile> storeFiles = primaryRegion.getStore(families[0]).getStorefiles();
       primaryRegion.getRegionFileSystem().removeStoreFiles(Bytes.toString(families[0]), storeFiles);
       Collection<StoreFileInfo> storeFileInfos = primaryRegion.getRegionFileSystem()
-          .getStoreFiles(families[0]);
+          .getStoreFiles(Bytes.toString(families[0]));
       Assert.assertTrue(storeFileInfos == null || storeFileInfos.isEmpty());
 
       verifyData(secondaryRegion, 0, 1000, cq, families);
@@ -7647,7 +7647,7 @@ public class TestHRegion {
             getCacheConfig() != null? getCacheConfig().shouldEvictOnClose(): true;
         for (Path newFile : newFiles) {
           // Create storefile around what we wrote with a reader on it.
-          HStoreFile sf = createStoreFileAndReader(newFile);
+          HStoreFile sf = storeEngine.createStoreFileAndReader(newFile);
           sf.closeStoreFile(evictOnClose);
           sfs.add(sf);
         }
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 2fdd627..bdee770 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
@@ -53,8 +53,8 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.function.IntBinaryOperator;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
@@ -315,7 +315,7 @@ public class TestHStore {
 
   /**
    * Verify that compression and data block encoding are respected by the
-   * Store.createWriterInTmp() method, used on store flush.
+   * createWriter method, used on store flush.
    */
   @Test
   public void testCreateWriter() throws Exception {
@@ -327,9 +327,11 @@ public class TestHStore {
         .build();
     init(name.getMethodName(), conf, hcd);
 
-    // Test createWriterInTmp()
-    StoreFileWriter writer =
-        store.createWriterInTmp(4, hcd.getCompressionType(), false, true, false, false);
+    // Test createWriter
+    StoreFileWriter writer = store.getStoreEngine()
+      .createWriter(CreateStoreFileWriterParams.create().maxKeyCount(4)
+        .compression(hcd.getCompressionType()).isCompaction(false).includeMVCCReadpoint(true)
+        .includesTag(false).shouldDropBehind(false));
     Path path = writer.getPath();
     writer.append(new KeyValue(row, family, qf1, Bytes.toBytes(1)));
     writer.append(new KeyValue(row, family, qf2, Bytes.toBytes(2)));
@@ -1027,19 +1029,19 @@ public class TestHStore {
     // add one more file
     addStoreFile();
 
-    HStore spiedStore = spy(store);
+    StoreEngine<?, ?, ?, ?> spiedStoreEngine = spy(store.getStoreEngine());
 
     // call first time after files changed
-    spiedStore.refreshStoreFiles();
+    spiedStoreEngine.refreshStoreFiles();
     assertEquals(2, this.store.getStorefilesCount());
-    verify(spiedStore, times(1)).replaceStoreFiles(any(), any());
+    verify(spiedStoreEngine, times(1)).replaceStoreFiles(any(), any());
 
     // call second time
-    spiedStore.refreshStoreFiles();
+    spiedStoreEngine.refreshStoreFiles();
 
     // ensure that replaceStoreFiles is not called, i.e, the times does not change, if files are not
     // refreshed,
-    verify(spiedStore, times(1)).replaceStoreFiles(any(), any());
+    verify(spiedStoreEngine, times(1)).replaceStoreFiles(any(), any());
   }
 
   private long countMemStoreScanner(StoreScanner scanner) {
@@ -1650,7 +1652,7 @@ public class TestHStore {
     // Do compaction
     MyThread thread = new MyThread(storeScanner);
     thread.start();
-    store.replaceStoreFiles(actualStorefiles, actualStorefiles1);
+    store.replaceStoreFiles(actualStorefiles, actualStorefiles1, false);
     thread.join();
     KeyValueHeap heap2 = thread.getHeap();
     assertFalse(heap.equals(heap2));
@@ -1729,8 +1731,10 @@ public class TestHStore {
   @Test
   public void testHFileContextSetWithCFAndTable() throws Exception {
     init(this.name.getMethodName());
-    StoreFileWriter writer = store.createWriterInTmp(10000L,
-        Compression.Algorithm.NONE, false, true, false, true);
+    StoreFileWriter writer = store.getStoreEngine()
+      .createWriter(CreateStoreFileWriterParams.create().maxKeyCount(10000L)
+        .compression(Compression.Algorithm.NONE).isCompaction(true).includeMVCCReadpoint(true)
+        .includesTag(false).shouldDropBehind(true));
     HFileContext hFileContext = writer.getHFileWriter().getFileContext();
     assertArrayEquals(family, hFileContext.getColumnFamily());
     assertArrayEquals(table, hFileContext.getTableName());
@@ -3277,7 +3281,8 @@ public class TestHStore {
         int currentCount = clearSnapshotCounter.incrementAndGet();
         if (currentCount == 1) {
           try {
-            if (store.lock.isWriteLockedByCurrentThread()) {
+            if (((ReentrantReadWriteLock) store.getStoreEngine().getLock())
+              .isWriteLockedByCurrentThread()) {
               shouldWait = false;
             }
             /**
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 be52892..814299a 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
@@ -245,7 +245,7 @@ public class TestRegionMergeTransactionOnCluster {
         TEST_UTIL.getConfiguration(), fs, tabledir, mergedRegionInfo);
       int count = 0;
       for(ColumnFamilyDescriptor colFamily : columnFamilies) {
-        count += hrfs.getStoreFiles(colFamily.getName()).size();
+        count += hrfs.getStoreFiles(colFamily.getNameAsString()).size();
       }
       ADMIN.compactRegion(mergedRegionInfo.getRegionName());
       // clean up the merged region store files
@@ -254,7 +254,7 @@ public class TestRegionMergeTransactionOnCluster {
       int newcount = 0;
       while (EnvironmentEdgeManager.currentTime() < timeout) {
         for(ColumnFamilyDescriptor colFamily : columnFamilies) {
-          newcount += hrfs.getStoreFiles(colFamily.getName()).size();
+          newcount += hrfs.getStoreFiles(colFamily.getNameAsString()).size();
         }
         if(newcount > count) {
           break;
@@ -273,7 +273,7 @@ public class TestRegionMergeTransactionOnCluster {
       while (EnvironmentEdgeManager.currentTime() < timeout) {
         int newcount1 = 0;
         for(ColumnFamilyDescriptor colFamily : columnFamilies) {
-          newcount1 += hrfs.getStoreFiles(colFamily.getName()).size();
+          newcount1 += hrfs.getStoreFiles(colFamily.getNameAsString()).size();
         }
         if(newcount1 <= 1) {
           break;
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 30a005d..daec3a2 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
@@ -26,7 +26,6 @@ import static org.mockito.Mockito.when;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collection;
 import java.util.List;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -98,7 +97,7 @@ public class TestStoreFileRefresherChore {
     }
 
     @Override
-    public Collection<StoreFileInfo> getStoreFiles(String familyName) throws IOException {
+    public List<StoreFileInfo> getStoreFiles(String familyName) throws IOException {
       if (fail) {
         throw new IOException("simulating FS failure");
       }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScannerClosure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScannerClosure.java
index f20691b..5372b84 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScannerClosure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScannerClosure.java
@@ -29,7 +29,6 @@ import java.util.NavigableSet;
 import java.util.Random;
 import java.util.TreeSet;
 import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -126,13 +125,12 @@ public class TestStoreScannerClosure {
       p.addColumn(fam, Bytes.toBytes("q1"), Bytes.toBytes("val"));
       region.put(p);
       HStore store = region.getStore(fam);
-      ReentrantReadWriteLock lock = store.lock;
       // use the lock to manually get a new memstore scanner. this is what
       // HStore#notifyChangedReadersObservers does under the lock.(lock is not needed here
       //since it is just a testcase).
-      lock.readLock().lock();
+      store.getStoreEngine().readLock();
       final List<KeyValueScanner> memScanners = store.memstore.getScanners(Long.MAX_VALUE);
-      lock.readLock().unlock();
+      store.getStoreEngine().readUnlock();
       Thread closeThread = new Thread() {
         public void run() {
           // close should be completed
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreEngine.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreEngine.java
index 7a93f89..498068b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreEngine.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreEngine.java
@@ -20,9 +20,9 @@ package org.apache.hadoop.hbase.regionserver;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyInt;
-import static org.mockito.Matchers.anyLong;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyInt;
+import static org.mockito.ArgumentMatchers.anyLong;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
@@ -36,7 +36,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.CellComparatorImpl;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequestImpl;
 import org.apache.hadoop.hbase.regionserver.compactions.StripeCompactionPolicy;
@@ -118,8 +118,10 @@ public class TestStripeStoreEngine {
   }
 
   private static TestStoreEngine createEngine(Configuration conf) throws Exception {
+    HRegion region = mock(HRegion.class);
     HStore store = mock(HStore.class);
-    when(store.getRegionInfo()).thenReturn(HRegionInfo.FIRST_META_REGIONINFO);
+    when(store.getRegionInfo()).thenReturn(RegionInfoBuilder.FIRST_META_REGIONINFO);
+    when(store.getHRegion()).thenReturn(region);
     CellComparatorImpl kvComparator = mock(CellComparatorImpl.class);
     return (TestStoreEngine)StoreEngine.create(store, conf, kvComparator);
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestDateTieredCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestDateTieredCompactor.java
index 92ba76d..9db1fad5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestDateTieredCompactor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestDateTieredCompactor.java
@@ -22,9 +22,6 @@ import static org.apache.hadoop.hbase.regionserver.compactions.TestCompactor.cre
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.ArgumentMatchers.anyBoolean;
-import static org.mockito.ArgumentMatchers.anyLong;
-import static org.mockito.ArgumentMatchers.anyString;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -41,14 +38,16 @@ import org.apache.hadoop.hbase.CellComparatorImpl;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.regionserver.CreateStoreFileWriterParams;
 import org.apache.hadoop.hbase.regionserver.HStore;
 import org.apache.hadoop.hbase.regionserver.HStoreFile;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
 import org.apache.hadoop.hbase.regionserver.ScanInfo;
 import org.apache.hadoop.hbase.regionserver.ScanType;
+import org.apache.hadoop.hbase.regionserver.StoreEngine;
 import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
 import org.apache.hadoop.hbase.regionserver.StoreUtils;
 import org.apache.hadoop.hbase.regionserver.compactions.TestCompactor.Scanner;
@@ -107,11 +106,10 @@ public class TestDateTieredCompactor {
     when(store.getScanInfo()).thenReturn(si);
     when(store.areWritesEnabled()).thenReturn(true);
     when(store.getFileSystem()).thenReturn(mock(FileSystem.class));
-    when(store.getRegionInfo()).thenReturn(new HRegionInfo(TABLE_NAME));
-    when(store.createWriterInTmp(anyLong(), any(), anyBoolean(),
-      anyBoolean(), anyBoolean(), anyBoolean())).thenAnswer(writers);
-    when(store.createWriterInTmp(anyLong(), any(), anyBoolean(),
-      anyBoolean(), anyBoolean(), anyBoolean(), anyLong(), anyString())).thenAnswer(writers);
+    when(store.getRegionInfo()).thenReturn(RegionInfoBuilder.newBuilder(TABLE_NAME).build());
+    StoreEngine storeEngine = mock(StoreEngine.class);
+    when(storeEngine.createWriter(any(CreateStoreFileWriterParams.class))).thenAnswer(writers);
+    when(store.getStoreEngine()).thenReturn(storeEngine);
     when(store.getComparator()).thenReturn(CellComparatorImpl.COMPARATOR);
     OptionalLong maxSequenceId = StoreUtils.getMaxSequenceIdInList(storefiles);
     when(store.getMaxSequenceId()).thenReturn(maxSequenceId);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactionPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactionPolicy.java
index d9f32fb..8e84797 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactionPolicy.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactionPolicy.java
@@ -30,7 +30,6 @@ import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.anyBoolean;
 import static org.mockito.ArgumentMatchers.anyInt;
 import static org.mockito.ArgumentMatchers.anyLong;
-import static org.mockito.ArgumentMatchers.anyString;
 import static org.mockito.ArgumentMatchers.argThat;
 import static org.mockito.ArgumentMatchers.eq;
 import static org.mockito.ArgumentMatchers.isNull;
@@ -58,6 +57,7 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.io.TimeRange;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.regionserver.BloomType;
+import org.apache.hadoop.hbase.regionserver.CreateStoreFileWriterParams;
 import org.apache.hadoop.hbase.regionserver.HStore;
 import org.apache.hadoop.hbase.regionserver.HStoreFile;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
@@ -65,6 +65,7 @@ import org.apache.hadoop.hbase.regionserver.ScanInfo;
 import org.apache.hadoop.hbase.regionserver.ScanType;
 import org.apache.hadoop.hbase.regionserver.ScannerContext;
 import org.apache.hadoop.hbase.regionserver.StoreConfigInformation;
+import org.apache.hadoop.hbase.regionserver.StoreEngine;
 import org.apache.hadoop.hbase.regionserver.StoreFileReader;
 import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
 import org.apache.hadoop.hbase.regionserver.StripeMultiFileWriter;
@@ -861,12 +862,9 @@ public class TestStripeCompactionPolicy {
     when(info.getRegionNameAsString()).thenReturn("testRegion");
     when(store.getColumnFamilyDescriptor()).thenReturn(col);
     when(store.getRegionInfo()).thenReturn(info);
-    when(
-      store.createWriterInTmp(anyLong(), any(), anyBoolean(),
-        anyBoolean(), anyBoolean(), anyBoolean())).thenAnswer(writers);
-    when(
-      store.createWriterInTmp(anyLong(), any(), anyBoolean(),
-        anyBoolean(), anyBoolean(), anyBoolean(), anyLong(), anyString())).thenAnswer(writers);
+    StoreEngine storeEngine = mock(StoreEngine.class);
+    when(storeEngine.createWriter(any(CreateStoreFileWriterParams.class))).thenAnswer(writers);
+    when(store.getStoreEngine()).thenReturn(storeEngine);
 
     Configuration conf = HBaseConfiguration.create();
     conf.setBoolean("hbase.regionserver.compaction.private.readers", usePrivateReaders);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactor.java
index 6e8b19f..c87b11f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactor.java
@@ -21,9 +21,6 @@ import static org.apache.hadoop.hbase.regionserver.StripeStoreFileManager.OPEN_K
 import static org.apache.hadoop.hbase.regionserver.compactions.TestCompactor.createDummyRequest;
 import static org.junit.Assert.assertEquals;
 import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.ArgumentMatchers.anyBoolean;
-import static org.mockito.ArgumentMatchers.anyLong;
-import static org.mockito.ArgumentMatchers.anyString;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -39,13 +36,15 @@ import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.regionserver.CreateStoreFileWriterParams;
 import org.apache.hadoop.hbase.regionserver.HStore;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
 import org.apache.hadoop.hbase.regionserver.ScanInfo;
 import org.apache.hadoop.hbase.regionserver.ScanType;
+import org.apache.hadoop.hbase.regionserver.StoreEngine;
 import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
 import org.apache.hadoop.hbase.regionserver.compactions.TestCompactor.Scanner;
 import org.apache.hadoop.hbase.regionserver.compactions.TestCompactor.StoreFileWritersCapture;
@@ -206,11 +205,10 @@ public class TestStripeCompactor {
     when(store.getScanInfo()).thenReturn(si);
     when(store.areWritesEnabled()).thenReturn(true);
     when(store.getFileSystem()).thenReturn(mock(FileSystem.class));
-    when(store.getRegionInfo()).thenReturn(new HRegionInfo(TABLE_NAME));
-    when(store.createWriterInTmp(anyLong(), any(), anyBoolean(),
-      anyBoolean(), anyBoolean(), anyBoolean())).thenAnswer(writers);
-    when(store.createWriterInTmp(anyLong(), any(), anyBoolean(),
-      anyBoolean(), anyBoolean(), anyBoolean(), anyLong(), anyString())).thenAnswer(writers);
+    when(store.getRegionInfo()).thenReturn(RegionInfoBuilder.newBuilder(TABLE_NAME).build());
+    StoreEngine storeEngine = mock(StoreEngine.class);
+    when(storeEngine.createWriter(any(CreateStoreFileWriterParams.class))).thenAnswer(writers);
+    when(store.getStoreEngine()).thenReturn(storeEngine);
     when(store.getComparator()).thenReturn(CellComparatorImpl.COMPARATOR);
 
     return new StripeCompactor(conf, store) {

[hbase] 08/15: HBASE-26280 Use store file tracker when snapshoting (#3685)

Posted by zh...@apache.org.
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

commit b812a277ab6baac42499a60b62dfefbe685fef41
Author: Duo Zhang <zh...@apache.org>
AuthorDate: Fri Sep 17 09:40:44 2021 +0800

    HBASE-26280 Use store file tracker when snapshoting (#3685)
    
    Signed-off-by: Wellington Chevreuil <wc...@apache.org>
    Reviewed-by: Josh Elser <el...@apache.org>
---
 .../assignment/MergeTableRegionsProcedure.java     |  5 +-
 .../assignment/SplitTableRegionProcedure.java      |  5 +-
 .../hbase/regionserver/HRegionFileSystem.java      | 10 +--
 .../storefiletracker/StoreFileTrackerFactory.java  | 17 ++---
 .../hadoop/hbase/snapshot/SnapshotManifest.java    | 42 ++++++------
 ...oneSnapshotFromClientCloneLinksAfterDelete.java |  4 +-
 .../hbase/client/TestMobSnapshotFromClient.java    |  7 +-
 .../hbase/client/TestSnapshotFromClient.java       | 34 +++++++---
 .../hadoop/hbase/regionserver/TestHStoreFile.java  |  6 +-
 .../hbase/snapshot/MobSnapshotTestingUtils.java    | 74 +++++++++++-----------
 .../hbase/snapshot/SnapshotTestingUtils.java       | 16 ++---
 11 files changed, 109 insertions(+), 111 deletions(-)

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 6e5151c..ab40207 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
@@ -613,9 +613,8 @@ public class MergeTableRegionsProcedure
     List<Path> mergedFiles = new ArrayList<>();
     for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) {
       String family = hcd.getNameAsString();
-      Configuration trackerConfig =
-        StoreFileTrackerFactory.mergeConfigurations(env.getMasterConfiguration(), htd, hcd);
-      StoreFileTracker tracker = StoreFileTrackerFactory.create(trackerConfig, family, regionFs);
+      StoreFileTracker tracker =
+        StoreFileTrackerFactory.create(env.getMasterConfiguration(), htd, hcd, regionFs);
       final Collection<StoreFileInfo> storeFiles = tracker.load();
       if (storeFiles != null && storeFiles.size() > 0) {
         final Configuration storeConfiguration =
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 aa0c938..effdba4 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
@@ -668,9 +668,8 @@ public class SplitTableRegionProcedure
         new HashMap<String, Collection<StoreFileInfo>>(htd.getColumnFamilyCount());
     for (ColumnFamilyDescriptor cfd : htd.getColumnFamilies()) {
       String family = cfd.getNameAsString();
-      Configuration trackerConfig = StoreFileTrackerFactory.
-        mergeConfigurations(env.getMasterConfiguration(), htd, htd.getColumnFamily(cfd.getName()));
-      StoreFileTracker tracker = StoreFileTrackerFactory.create(trackerConfig, family, regionFs);
+      StoreFileTracker tracker =
+        StoreFileTrackerFactory.create(env.getMasterConfiguration(), htd, cfd, regionFs);
       Collection<StoreFileInfo> sfis = tracker.load();
       if (sfis == null) {
         continue;
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 e78d8ad..8110025 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
@@ -598,7 +598,6 @@ public class HRegionFileSystem {
    * to the proper location in the filesystem.
    *
    * @param regionInfo daughter {@link org.apache.hadoop.hbase.client.RegionInfo}
-   * @throws IOException
    */
   public Path commitDaughterRegion(final RegionInfo regionInfo, List<Path> allRegionFiles,
       MasterProcedureEnv env) throws IOException {
@@ -625,12 +624,8 @@ public class HRegionFileSystem {
     Map<String, List<StoreFileInfo>> fileInfoMap = new HashMap<>();
     for(Path file : allFiles) {
       String familyName = file.getParent().getName();
-      trackerMap.computeIfAbsent(familyName, t -> {
-        Configuration config = StoreFileTrackerFactory.mergeConfigurations(conf, tblDesc,
-          tblDesc.getColumnFamily(Bytes.toBytes(familyName)));
-        return StoreFileTrackerFactory.
-          create(config, familyName, regionFs);
-      });
+      trackerMap.computeIfAbsent(familyName, t -> StoreFileTrackerFactory.create(conf, tblDesc,
+        tblDesc.getColumnFamily(Bytes.toBytes(familyName)), regionFs));
       fileInfoMap.computeIfAbsent(familyName, l -> new ArrayList<>());
       List<StoreFileInfo> infos = fileInfoMap.get(familyName);
       infos.add(new StoreFileInfo(conf, fs, file, true));
@@ -676,7 +671,6 @@ public class HRegionFileSystem {
    *                    this method is invoked on the Master side, then the RegionSplitPolicy will
    *                    NOT have a reference to a Region.
    * @return Path to created reference.
-   * @throws IOException
    */
   public Path splitStoreFile(RegionInfo hri, String familyName, HStoreFile f, byte[] splitRow,
       boolean top, RegionSplitPolicy splitPolicy) throws IOException {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java
index 90704fe..b586027 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java
@@ -22,13 +22,11 @@ import java.util.Map;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
-import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
 import org.apache.hadoop.hbase.regionserver.StoreContext;
 import org.apache.hadoop.hbase.regionserver.StoreUtils;
-import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ReflectionUtils;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
@@ -113,16 +111,15 @@ public final class StoreFileTrackerFactory {
    * Used at master side when splitting/merging regions, as we do not have a Store, thus no
    * StoreContext at master side.
    */
-  public static StoreFileTracker create(Configuration conf, String family,
-    HRegionFileSystem regionFs) {
-    ColumnFamilyDescriptorBuilder fDescBuilder =
-      ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(family));
-    StoreContext ctx = StoreContext.getBuilder().withColumnFamilyDescriptor(fDescBuilder.build())
-      .withRegionFileSystem(regionFs).build();
-    return StoreFileTrackerFactory.create(conf, true, ctx);
+  public static StoreFileTracker create(Configuration conf, TableDescriptor td,
+    ColumnFamilyDescriptor cfd, HRegionFileSystem regionFs) {
+    StoreContext ctx =
+      StoreContext.getBuilder().withColumnFamilyDescriptor(cfd).withRegionFileSystem(regionFs)
+        .withFamilyStoreDirectoryPath(regionFs.getStoreDir(cfd.getNameAsString())).build();
+    return StoreFileTrackerFactory.create(mergeConfigurations(conf, td, cfd), true, ctx);
   }
 
-  public static Configuration mergeConfigurations(Configuration global, TableDescriptor table,
+  private static Configuration mergeConfigurations(Configuration global, TableDescriptor table,
     ColumnFamilyDescriptor family) {
     return StoreUtils.createStoreConfiguration(global, table, family);
   }
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 79e587a..368e72b 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
@@ -47,7 +47,8 @@ import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
 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.regionserver.storefiletracker.StoreFileTracker;
+import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory;
 import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.FSTableDescriptors;
 import org.apache.hadoop.hbase.util.Threads;
@@ -291,8 +292,8 @@ public final class SnapshotManifest {
     addRegion(tableDir, regionInfo, visitor);
   }
 
-  protected void addRegion(final Path tableDir, final RegionInfo regionInfo, RegionVisitor visitor)
-      throws IOException {
+  protected void addRegion(Path tableDir, RegionInfo regionInfo, RegionVisitor visitor)
+    throws IOException {
     boolean isMobRegion = MobUtils.isMobRegionInfo(regionInfo);
     try {
       Path baseDir = tableDir;
@@ -300,8 +301,8 @@ public final class SnapshotManifest {
       if (isMobRegion) {
         baseDir = CommonFSUtils.getTableDir(MobUtils.getMobHome(conf), regionInfo.getTable());
       }
-      HRegionFileSystem regionFs = HRegionFileSystem.openRegionFromFileSystem(conf, rootFs,
-        baseDir, regionInfo, true);
+      HRegionFileSystem regionFs =
+        HRegionFileSystem.openRegionFromFileSystem(conf, rootFs, baseDir, regionInfo, true);
       monitor.rethrowException();
 
       // 1. dump region meta info into the snapshot directory
@@ -317,26 +318,19 @@ public final class SnapshotManifest {
       // in batches and may miss files being added/deleted. This could be more robust (iteratively
       // checking to see if we have all the files until we are sure), but the limit is currently
       // 1000 files/batch, far more than the number of store files under a single column family.
-      Collection<String> familyNames = regionFs.getFamilies();
-      if (familyNames != null) {
-        for (String familyName: familyNames) {
-          Object familyData = visitor.familyOpen(regionData, Bytes.toBytes(familyName));
-          monitor.rethrowException();
-
-          Collection<StoreFileInfo> storeFiles = regionFs.getStoreFiles(familyName);
-          if (storeFiles == null) {
-            if (LOG.isDebugEnabled()) {
-              LOG.debug("No files under family: " + familyName);
-            }
-            continue;
-          }
-
-          // 2.1. build the snapshot reference for the store
-          // iterate through all the store's files and create "references".
-          addReferenceFiles(visitor, regionData, familyData, storeFiles, false);
-
-          visitor.familyClose(regionData, familyData);
+      for (ColumnFamilyDescriptor cfd : htd.getColumnFamilies()) {
+        Object familyData = visitor.familyOpen(regionData, cfd.getName());
+        monitor.rethrowException();
+        StoreFileTracker tracker = StoreFileTrackerFactory.create(conf, htd, cfd, regionFs);
+        List<StoreFileInfo> storeFiles = tracker.load();
+        if (storeFiles.isEmpty()) {
+          LOG.debug("No files under family: {}", cfd.getNameAsString());
+          continue;
         }
+        // 2.1. build the snapshot reference for the store
+        // iterate through all the store's files and create "references".
+        addReferenceFiles(visitor, regionData, familyData, storeFiles, false);
+        visitor.familyClose(regionData, familyData);
       }
       visitor.regionClose(regionData);
     } catch (IOException e) {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobCloneSnapshotFromClientCloneLinksAfterDelete.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobCloneSnapshotFromClientCloneLinksAfterDelete.java
index 699c086..3c24911 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobCloneSnapshotFromClientCloneLinksAfterDelete.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobCloneSnapshotFromClientCloneLinksAfterDelete.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.coprocessor.RegionObserver;
 import org.apache.hadoop.hbase.master.cleaner.TimeToLiveHFileCleaner;
 import org.apache.hadoop.hbase.mob.MobConstants;
 import org.apache.hadoop.hbase.regionserver.FlushLifeCycleTracker;
+import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory;
 import org.apache.hadoop.hbase.snapshot.MobSnapshotTestingUtils;
 import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
@@ -92,7 +93,8 @@ public class TestMobCloneSnapshotFromClientCloneLinksAfterDelete
   @Override
   protected void createTable() throws IOException, InterruptedException {
     MobSnapshotTestingUtils.createMobTable(TEST_UTIL, tableName,
-      SnapshotTestingUtils.getSplitKeys(), getNumReplicas(), DelayFlushCoprocessor.class.getName(),
+      SnapshotTestingUtils.getSplitKeys(), getNumReplicas(),
+      StoreFileTrackerFactory.Trackers.DEFAULT.name(), DelayFlushCoprocessor.class.getName(),
       FAMILY);
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobSnapshotFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobSnapshotFromClient.java
index cdc41b0..0695be1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobSnapshotFromClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobSnapshotFromClient.java
@@ -26,8 +26,6 @@ import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.experimental.categories.Category;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
  * Test create/using/deleting snapshots from the client
@@ -41,8 +39,6 @@ public class TestMobSnapshotFromClient extends TestSnapshotFromClient {
   public static final HBaseClassTestRule CLASS_RULE =
       HBaseClassTestRule.forClass(TestMobSnapshotFromClient.class);
 
-  private static final Logger LOG = LoggerFactory.getLogger(TestMobSnapshotFromClient.class);
-
   /**
    * Setup the config for the cluster
    * @throws Exception on failure
@@ -60,6 +56,7 @@ public class TestMobSnapshotFromClient extends TestSnapshotFromClient {
 
   @Override
   protected void createTable() throws Exception {
-    MobSnapshotTestingUtils.createMobTable(UTIL, TABLE_NAME, getNumReplicas(), TEST_FAM);
+    MobSnapshotTestingUtils.createMobTable(UTIL, TABLE_NAME, getNumReplicas(), trackerImpl.name(),
+      TEST_FAM);
   }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java
index 65704ce..76356fe 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
@@ -23,6 +23,7 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
 import java.util.regex.Pattern;
 import org.apache.hadoop.conf.Configuration;
@@ -33,9 +34,11 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.TableNameTestRule;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy;
+import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory;
 import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
 import org.apache.hadoop.hbase.snapshot.SnapshotDoesNotExistException;
 import org.apache.hadoop.hbase.snapshot.SnapshotManifestV1;
@@ -52,7 +55,10 @@ import org.junit.ClassRule;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-import org.junit.rules.TestName;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameter;
+import org.junit.runners.Parameterized.Parameters;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -65,7 +71,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
  * <p>
  * This is an end-to-end test for the snapshot utility
  */
-@Category({LargeTests.class, ClientTests.class})
+@RunWith(Parameterized.class)
+@Category({ LargeTests.class, ClientTests.class })
 public class TestSnapshotFromClient {
 
   @ClassRule
@@ -83,7 +90,16 @@ public class TestSnapshotFromClient {
   private static final Pattern MATCH_ALL = Pattern.compile(".*");
 
   @Rule
-  public TestName name = new TestName();
+  public TableNameTestRule name = new TableNameTestRule();
+
+  @Parameter
+  public StoreFileTrackerFactory.Trackers trackerImpl;
+
+  @Parameters(name = "{index}: tracker={0}")
+  public static List<Object[]> params() {
+    return Arrays.asList(new Object[] { StoreFileTrackerFactory.Trackers.DEFAULT },
+      new Object[] { StoreFileTrackerFactory.Trackers.FILE });
+  }
 
   /**
    * Setup the config for the cluster
@@ -110,7 +126,6 @@ public class TestSnapshotFromClient {
     conf.setBoolean(SnapshotManager.HBASE_SNAPSHOT_ENABLED, true);
     conf.set(HConstants.HBASE_REGION_SPLIT_POLICY_KEY,
         ConstantSizeRegionSplitPolicy.class.getName());
-
   }
 
   @Before
@@ -119,9 +134,10 @@ public class TestSnapshotFromClient {
   }
 
   protected void createTable() throws Exception {
-    HTableDescriptor htd = new HTableDescriptor(TABLE_NAME);
-    htd.setRegionReplication(getNumReplicas());
-    UTIL.createTable(htd, new byte[][]{TEST_FAM}, null);
+    TableDescriptor htd =
+      TableDescriptorBuilder.newBuilder(TABLE_NAME).setRegionReplication(getNumReplicas())
+        .setValue(StoreFileTrackerFactory.TRACKER_IMPL, trackerImpl.name()).build();
+    UTIL.createTable(htd, new byte[][] { TEST_FAM }, null);
   }
 
   protected int getNumReplicas() {
@@ -326,7 +342,7 @@ public class TestSnapshotFromClient {
   @Test
   public void testListTableSnapshots() throws Exception {
     Admin admin = null;
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     try {
       admin = UTIL.getAdmin();
 
@@ -411,7 +427,7 @@ public class TestSnapshotFromClient {
   @Test
   public void testDeleteTableSnapshots() throws Exception {
     Admin admin = null;
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
     try {
       admin = UTIL.getAdmin();
 
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 efe6fce..59a0df3 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
@@ -53,6 +53,7 @@ import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
 import org.apache.hadoop.hbase.io.HFileLink;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
@@ -1103,10 +1104,9 @@ public class TestHStoreFile extends HBaseTestCase {
     when(mockEnv.getMasterConfiguration()).thenReturn(new Configuration());
     TableDescriptors mockTblDescs = mock(TableDescriptors.class);
     when(mockServices.getTableDescriptors()).thenReturn(mockTblDescs);
-    TableDescriptor mockTblDesc = mock(TableDescriptor.class);
+    TableDescriptor mockTblDesc = TableDescriptorBuilder.newBuilder(hri.getTable())
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(family)).build();
     when(mockTblDescs.get(any())).thenReturn(mockTblDesc);
-    ColumnFamilyDescriptor mockCfDesc = mock(ColumnFamilyDescriptor.class);
-    when(mockTblDesc.getColumnFamily(any())).thenReturn(mockCfDesc);
     Path regionDir = regionFs.commitDaughterRegion(hri, splitFiles, mockEnv);
     return new Path(new Path(regionDir, family), path.getName());
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/MobSnapshotTestingUtils.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/MobSnapshotTestingUtils.java
index 9ee4336..4fb1ae2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/MobSnapshotTestingUtils.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/MobSnapshotTestingUtils.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.regionserver.BloomType;
+import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.Assert;
 
@@ -45,29 +46,40 @@ public class MobSnapshotTestingUtils {
   /**
    * Create the Mob Table.
    */
-  public static void createMobTable(final HBaseTestingUtility util,
-      final TableName tableName, int regionReplication,
-      final byte[]... families) throws IOException, InterruptedException {
-    createMobTable(util, tableName, SnapshotTestingUtils.getSplitKeys(),
-      regionReplication, families);
+  public static void createMobTable(final HBaseTestingUtility util, final TableName tableName,
+    int regionReplication, final byte[]... families) throws IOException, InterruptedException {
+    createMobTable(util, tableName, SnapshotTestingUtils.getSplitKeys(), regionReplication,
+      StoreFileTrackerFactory.Trackers.DEFAULT.name(), families);
+  }
+
+  public static void createMobTable(final HBaseTestingUtility util, final TableName tableName,
+    int regionReplication, String storeFileTracker, final byte[]... families)
+    throws IOException, InterruptedException {
+    createMobTable(util, tableName, SnapshotTestingUtils.getSplitKeys(), regionReplication,
+      storeFileTracker, families);
   }
 
-  public static void createPreSplitMobTable(final HBaseTestingUtility util,
-      final TableName tableName, int nRegions, final byte[]... families)
-      throws IOException, InterruptedException {
-    createMobTable(util, tableName, SnapshotTestingUtils.getSplitKeys(nRegions),
-      1, families);
+  public static void createPreSplitMobTable(final HBaseTestingUtility util, final TableName tableName,
+    int nRegions, final byte[]... families) throws IOException, InterruptedException {
+    createMobTable(util, tableName, SnapshotTestingUtils.getSplitKeys(nRegions), 1, families);
+  }
+
+  public static void createMobTable(final HBaseTestingUtility util, final TableName tableName,
+    final byte[][] splitKeys, int regionReplication, final byte[]... families)
+    throws IOException, InterruptedException {
+    createMobTable(util, tableName, splitKeys, regionReplication,
+      StoreFileTrackerFactory.Trackers.DEFAULT.name(), families);
   }
 
   public static void createMobTable(final HBaseTestingUtility util, final TableName tableName,
-      final byte[][] splitKeys, int regionReplication, final byte[]... families)
-      throws IOException, InterruptedException {
-    createMobTable(util, tableName, splitKeys, regionReplication, null, families);
+    final byte[][] splitKeys, int regionReplication, String storeFileTracker,
+    final byte[]... families) throws IOException, InterruptedException {
+    createMobTable(util, tableName, splitKeys, regionReplication, storeFileTracker, null, families);
   }
 
-  public static void createMobTable(HBaseTestingUtility util, TableName tableName,
-      byte[][] splitKeys, int regionReplication, String cpClassName, byte[]... families)
-      throws IOException, InterruptedException {
+  public static void createMobTable(HBaseTestingUtility util, TableName tableName, byte[][] splitKeys,
+    int regionReplication, String storeFileTracker, String cpClassName, byte[]... families)
+    throws IOException, InterruptedException {
     TableDescriptorBuilder builder =
       TableDescriptorBuilder.newBuilder(tableName).setRegionReplication(regionReplication);
     for (byte[] family : families) {
@@ -77,6 +89,7 @@ public class MobSnapshotTestingUtils {
     if (!StringUtils.isBlank(cpClassName)) {
       builder.setCoprocessor(cpClassName);
     }
+    builder.setValue(StoreFileTrackerFactory.TRACKER_IMPL, storeFileTracker);
     util.getAdmin().createTable(builder.build(), splitKeys);
     SnapshotTestingUtils.waitForTableToBeOnline(util, tableName);
     assertEquals((splitKeys.length + 1) * regionReplication,
@@ -85,15 +98,10 @@ public class MobSnapshotTestingUtils {
 
   /**
    * Create a Mob table.
-   *
-   * @param util
-   * @param tableName
-   * @param families
    * @return An Table instance for the created table.
-   * @throws IOException
    */
-  public static Table createMobTable(final HBaseTestingUtility util,
-      final TableName tableName, final byte[]... families) throws IOException {
+  public static Table createMobTable(final HBaseTestingUtility util, final TableName tableName,
+    final byte[]... families) throws IOException {
     TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName);
     for (byte[] family : families) {
       // Disable blooms (they are on by default as of 0.95) but we disable them
@@ -102,10 +110,7 @@ public class MobSnapshotTestingUtils {
       // and blooms being
       // on is interfering.
       builder.setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(family)
-              .setBloomFilterType(BloomType.NONE)
-              .setMobEnabled(true)
-              .setMobThreshold(0L)
-              .build());
+        .setBloomFilterType(BloomType.NONE).setMobEnabled(true).setMobThreshold(0L).build());
     }
     util.getAdmin().createTable(builder.build());
     // HBaseAdmin only waits for regions to appear in hbase:meta we should wait
@@ -135,8 +140,8 @@ public class MobSnapshotTestingUtils {
     }
   }
 
-  public static void verifyMobRowCount(final HBaseTestingUtility util,
-      final TableName tableName, long expectedRows) throws IOException {
+  public static void verifyMobRowCount(final HBaseTestingUtility util, final TableName tableName,
+    long expectedRows) throws IOException {
 
     Table table = ConnectionFactory.createConnection(util.getConfiguration()).getTable(tableName);
     try {
@@ -156,13 +161,10 @@ public class MobSnapshotTestingUtils {
 
     @Override
     public TableDescriptor createHtd(final String tableName) {
-      return TableDescriptorBuilder.newBuilder(TableName.valueOf(tableName))
-              .setColumnFamily(ColumnFamilyDescriptorBuilder
-                  .newBuilder(Bytes.toBytes(TEST_FAMILY))
-                  .setMobEnabled(true)
-                  .setMobThreshold(0L)
-                  .build())
-              .build();
+      return TableDescriptorBuilder
+        .newBuilder(TableName.valueOf(tableName)).setColumnFamily(ColumnFamilyDescriptorBuilder
+          .newBuilder(Bytes.toBytes(TEST_FAMILY)).setMobEnabled(true).setMobThreshold(0L).build())
+        .build();
     }
   }
 }
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 509842b..a19bf4f 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
@@ -509,9 +509,8 @@ public final class SnapshotTestingUtils {
         this.desc = desc;
         this.tableRegions = tableRegions;
         this.snapshotDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(desc, rootDir, conf);
-        new FSTableDescriptors(conf)
-          .createTableDescriptorForTableDirectory(this.snapshotDir.getFileSystem(conf),
-            snapshotDir, htd, false);
+        FSTableDescriptors.createTableDescriptorForTableDirectory(
+          this.snapshotDir.getFileSystem(conf), snapshotDir, htd, false);
       }
 
       public TableDescriptor getTableDescriptor() {
@@ -531,15 +530,13 @@ public final class SnapshotTestingUtils {
       }
 
       public Path[] addRegionV1() throws IOException {
-        return addRegion(desc.toBuilder()
-                          .setVersion(SnapshotManifestV1.DESCRIPTOR_VERSION)
-                          .build());
+        return addRegion(
+          desc.toBuilder().setVersion(SnapshotManifestV1.DESCRIPTOR_VERSION).build());
       }
 
       public Path[] addRegionV2() throws IOException {
-        return addRegion(desc.toBuilder()
-                          .setVersion(SnapshotManifestV2.DESCRIPTOR_VERSION)
-                          .build());
+        return addRegion(
+          desc.toBuilder().setVersion(SnapshotManifestV2.DESCRIPTOR_VERSION).build());
       }
 
       private Path[] addRegion(final SnapshotProtos.SnapshotDescription desc) throws IOException {
@@ -550,6 +547,7 @@ public final class SnapshotTestingUtils {
         RegionData regionData = tableRegions[this.snapshotted++];
         ForeignExceptionDispatcher monitor = new ForeignExceptionDispatcher(desc.getName());
         SnapshotManifest manifest = SnapshotManifest.create(conf, fs, snapshotDir, desc, monitor);
+        manifest.addTableDescriptor(htd);
         manifest.addRegion(regionData.tableDir, regionData.hri);
         return regionData.files;
       }

[hbase] 11/15: HBASE-26328 Clone snapshot doesn't load reference files into FILE SFT impl (#3749)

Posted by zh...@apache.org.
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

commit 80b04229a2f0f54826c385641053d3e59ab80a7c
Author: Wellington Ramos Chevreuil <wc...@apache.org>
AuthorDate: Fri Oct 22 16:56:15 2021 +0100

    HBASE-26328 Clone snapshot doesn't load reference files into FILE SFT impl (#3749)
    
    Signed-off-by: Duo Zhang <zh...@apache.org>
---
 .../java/org/apache/hadoop/hbase/io/HFileLink.java | 63 ++++++--------
 .../master/procedure/CloneSnapshotProcedure.java   | 52 +++---------
 .../storefiletracker/StoreFileTrackerBase.java     |  1 +
 .../hbase/snapshot/RestoreSnapshotHelper.java      | 95 ++++++++++++++++++----
 .../TestCloneSnapshotProcedureFileBasedSFT.java    | 42 ++++++++++
 5 files changed, 157 insertions(+), 96 deletions(-)

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 74836ce..fbed724 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
@@ -201,7 +201,6 @@ public class HFileLink extends FileLink {
     return isHFileLink(path.getName());
   }
 
-
   /**
    * @param fileName File name to check.
    * @return True if the path is a HFileLink.
@@ -322,10 +321,10 @@ public class HFileLink extends FileLink {
    * @param dstFamilyPath - Destination path (table/region/cf/)
    * @param hfileRegionInfo - Linked HFile Region Info
    * @param hfileName - Linked HFile name
-   * @return true if the file is created, otherwise the file exists.
-   * @throws IOException on file or parent directory creation failure
+   * @return the file link name.
+   * @throws IOException on file or parent directory creation failure.
    */
-  public static boolean create(final Configuration conf, final FileSystem fs,
+  public static String create(final Configuration conf, final FileSystem fs,
       final Path dstFamilyPath, final RegionInfo hfileRegionInfo,
       final String hfileName) throws IOException {
     return create(conf, fs, dstFamilyPath, hfileRegionInfo, hfileName, true);
@@ -343,10 +342,10 @@ public class HFileLink extends FileLink {
    * @param hfileRegionInfo - Linked HFile Region Info
    * @param hfileName - Linked HFile name
    * @param createBackRef - Whether back reference should be created. Defaults to true.
-   * @return true if the file is created, otherwise the file exists.
-   * @throws IOException on file or parent directory creation failure
+   * @return the file link name.
+   * @throws IOException on file or parent directory creation failure.
    */
-  public static boolean create(final Configuration conf, final FileSystem fs,
+  public static String create(final Configuration conf, final FileSystem fs,
       final Path dstFamilyPath, final RegionInfo hfileRegionInfo,
       final String hfileName, final boolean createBackRef) throws IOException {
     TableName linkedTable = hfileRegionInfo.getTable();
@@ -366,17 +365,18 @@ public class HFileLink extends FileLink {
    * @param linkedTable - Linked Table Name
    * @param linkedRegion - Linked Region Name
    * @param hfileName - Linked HFile name
-   * @return true if the file is created, otherwise the file exists.
-   * @throws IOException on file or parent directory creation failure
+   * @return the file link name.
+   * @throws IOException on file or parent directory creation failure.
    */
-  public static boolean create(final Configuration conf, final FileSystem fs,
+  public static String create(final Configuration conf, final FileSystem fs,
       final Path dstFamilyPath, final TableName linkedTable, final String linkedRegion,
       final String hfileName) throws IOException {
     return create(conf, fs, dstFamilyPath, linkedTable, linkedRegion, hfileName, true);
   }
 
   /**
-   * Create a new HFileLink
+   * Create a new HFileLink. In the event of link creation failure, this method throws an
+   * IOException, so that the calling upper laying can decide on how to proceed with this.
    *
    * <p>It also adds a back-reference to the hfile back-reference directory
    * to simplify the reference-count and the cleaning process.
@@ -388,10 +388,10 @@ public class HFileLink extends FileLink {
    * @param linkedRegion - Linked Region Name
    * @param hfileName - Linked HFile name
    * @param createBackRef - Whether back reference should be created. Defaults to true.
-   * @return true if the file is created, otherwise the file exists.
-   * @throws IOException on file or parent directory creation failure
+   * @return the file link name.
+   * @throws IOException on file or parent directory creation failure.
    */
-  public static boolean create(final Configuration conf, final FileSystem fs,
+  public static String create(final Configuration conf, final FileSystem fs,
       final Path dstFamilyPath, final TableName linkedTable, final String linkedRegion,
       final String hfileName, final boolean createBackRef) throws IOException {
     String familyName = dstFamilyPath.getName();
@@ -417,10 +417,10 @@ public class HFileLink extends FileLink {
    * @param linkedRegion - Linked Region Name
    * @param hfileName - Linked HFile name
    * @param createBackRef - Whether back reference should be created. Defaults to true.
-   * @return true if the file is created, otherwise the file exists.
+   * @return the file link name.
    * @throws IOException on file or parent directory creation failure
    */
-  public static boolean create(final Configuration conf, final FileSystem fs,
+  public static String create(final Configuration conf, final FileSystem fs,
       final Path dstFamilyPath, final String familyName, final String dstTableName,
       final String dstRegionName, final TableName linkedTable, final String linkedRegion,
       final String hfileName, final boolean createBackRef) throws IOException {
@@ -444,7 +444,9 @@ public class HFileLink extends FileLink {
     }
     try {
       // Create the link
-      return fs.createNewFile(new Path(dstFamilyPath, name));
+      if (fs.createNewFile(new Path(dstFamilyPath, name))) {
+        return name;
+      }
     } catch (IOException e) {
       LOG.error("couldn't create the link=" + name + " for " + dstFamilyPath, e);
       // Revert the reference if the link creation failed
@@ -453,25 +455,8 @@ public class HFileLink extends FileLink {
       }
       throw e;
     }
-  }
-
-  /**
-   * Create a new HFileLink starting from a hfileLink name
-   *
-   * <p>It also adds a back-reference to the hfile back-reference directory
-   * to simplify the reference-count and the cleaning process.
-   *
-   * @param conf {@link Configuration} to read for the archive directory name
-   * @param fs {@link FileSystem} on which to write the HFileLink
-   * @param dstFamilyPath - Destination path (table/region/cf/)
-   * @param hfileLinkName - HFileLink name (it contains hfile-region-table)
-   * @return true if the file is created, otherwise the file exists.
-   * @throws IOException on file or parent directory creation failure
-   */
-  public static boolean createFromHFileLink(final Configuration conf, final FileSystem fs,
-      final Path dstFamilyPath, final String hfileLinkName)
-          throws IOException {
-    return createFromHFileLink(conf, fs, dstFamilyPath, hfileLinkName, true);
+    throw new IOException("File link=" + name + " already exists under " +
+      dstFamilyPath + " folder.");
   }
 
   /**
@@ -485,10 +470,10 @@ public class HFileLink extends FileLink {
    * @param dstFamilyPath - Destination path (table/region/cf/)
    * @param hfileLinkName - HFileLink name (it contains hfile-region-table)
    * @param createBackRef - Whether back reference should be created. Defaults to true.
-   * @return true if the file is created, otherwise the file exists.
-   * @throws IOException on file or parent directory creation failure
+   * @return the file link name.
+   * @throws IOException on file or parent directory creation failure.
    */
-  public static boolean createFromHFileLink(final Configuration conf, final FileSystem fs,
+  public static String createFromHFileLink(final Configuration conf, final FileSystem fs,
       final Path dstFamilyPath, final String hfileLinkName, final boolean createBackRef)
           throws IOException {
     Matcher m = LINK_NAME_PATTERN.matcher(hfileLinkName);
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 8157af9..7157fbf 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
@@ -41,7 +41,6 @@ import org.apache.hadoop.hbase.master.MetricsSnapshot;
 import org.apache.hadoop.hbase.master.RegionState;
 import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 import org.apache.hadoop.hbase.master.procedure.CreateTableProcedure.CreateHdfsRegions;
-import org.apache.hadoop.hbase.mob.MobUtils;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
 import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
@@ -459,56 +458,25 @@ public class CloneSnapshotProcedure
     List<RegionInfo> newRegions,
     final CreateHdfsRegions hdfsRegionHandler) throws IOException {
     final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
-    final Path tempdir = mfs.getTempDir();
 
     // 1. Create Table Descriptor
     // using a copy of descriptor, table will be created enabling first
-    final Path tempTableDir = CommonFSUtils.getTableDir(tempdir, tableDescriptor.getTableName());
-    if (CommonFSUtils.isExists(mfs.getFileSystem(), tempTableDir)) {
+    final Path tableDir = CommonFSUtils.getTableDir(mfs.getRootDir(),
+      tableDescriptor.getTableName());
+    if (CommonFSUtils.isExists(mfs.getFileSystem(), tableDir)) {
       // if the region dirs exist, will cause exception and unlimited retry (see HBASE-24546)
-      LOG.warn("temp table dir already exists on disk: {}, will be deleted.", tempTableDir);
-      CommonFSUtils.deleteDirectory(mfs.getFileSystem(), tempTableDir);
+      LOG.warn("temp table dir already exists on disk: {}, will be deleted.", tableDir);
+      CommonFSUtils.deleteDirectory(mfs.getFileSystem(), tableDir);
     }
-    ((FSTableDescriptors) (env.getMasterServices().getTableDescriptors()))
-      .createTableDescriptorForTableDirectory(tempTableDir,
-        TableDescriptorBuilder.newBuilder(tableDescriptor).build(), false);
+    ((FSTableDescriptors)(env.getMasterServices().getTableDescriptors()))
+      .createTableDescriptorForTableDirectory(tableDir,
+              TableDescriptorBuilder.newBuilder(tableDescriptor).build(), false);
 
     // 2. Create Regions
     newRegions = hdfsRegionHandler.createHdfsRegions(
-      env, tempdir, tableDescriptor.getTableName(), newRegions);
-
-    // 3. Move Table temp directory to the hbase root location
-    CreateTableProcedure.moveTempDirectoryToHBaseRoot(env, tableDescriptor, tempTableDir);
-    // Move Table temp mob directory to the hbase root location
-    Path tempMobTableDir = MobUtils.getMobTableDir(tempdir, tableDescriptor.getTableName());
-    if (mfs.getFileSystem().exists(tempMobTableDir)) {
-      moveTempMobDirectoryToHBaseRoot(mfs, tableDescriptor, tempMobTableDir);
-    }
-    return newRegions;
-  }
+      env, mfs.getRootDir(), tableDescriptor.getTableName(), newRegions);
 
-  /**
-   * Move table temp mob directory to the hbase root location
-   * @param mfs The master file system
-   * @param tableDescriptor The table to operate on
-   * @param tempMobTableDir The temp mob directory of table
-   * @throws IOException If failed to move temp mob dir to hbase root dir
-   */
-  private void moveTempMobDirectoryToHBaseRoot(final MasterFileSystem mfs,
-      final TableDescriptor tableDescriptor, final Path tempMobTableDir) throws IOException {
-    FileSystem fs = mfs.getFileSystem();
-    final Path tableMobDir =
-        MobUtils.getMobTableDir(mfs.getRootDir(), tableDescriptor.getTableName());
-    if (!fs.delete(tableMobDir, true) && fs.exists(tableMobDir)) {
-      throw new IOException("Couldn't delete mob table " + tableMobDir);
-    }
-    if (!fs.exists(tableMobDir.getParent())) {
-      fs.mkdirs(tableMobDir.getParent());
-    }
-    if (!fs.rename(tempMobTableDir, tableMobDir)) {
-      throw new IOException("Unable to move mob table from temp=" + tempMobTableDir
-          + " to hbase root=" + tableMobDir);
-    }
+    return newRegions;
   }
 
   /**
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java
index d3c5859..b6de32b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java
@@ -21,6 +21,7 @@ import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTra
 
 import java.io.IOException;
 import java.util.Collection;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
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 bd5388c..7c75e46 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
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.hbase.snapshot;
 
+import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.TRACKER_IMPL;
+
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
@@ -53,7 +55,10 @@ import org.apache.hadoop.hbase.monitoring.MonitoredTask;
 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
+import org.apache.hadoop.hbase.regionserver.StoreContext;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
+import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTracker;
+import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory;
 import org.apache.hadoop.hbase.security.access.AccessControlClient;
 import org.apache.hadoop.hbase.security.access.Permission;
 import org.apache.hadoop.hbase.security.access.ShadedAccessControlUtil;
@@ -500,18 +505,28 @@ public class RestoreSnapshotHelper {
     String tableName = tableDesc.getTableName().getNameAsString();
     final String snapshotName = snapshotDesc.getName();
 
+    Path regionPath = new Path(tableDir, regionInfo.getEncodedName());
+    HRegionFileSystem regionFS = (fs.exists(regionPath)) ?
+      HRegionFileSystem.openRegionFromFileSystem(conf, fs, tableDir, regionInfo, false) :
+      HRegionFileSystem.createRegionOnFileSystem(conf, fs, tableDir, regionInfo);
+
     // Restore families present in the table
     for (Path familyDir: FSUtils.getFamilyDirs(fs, regionDir)) {
       byte[] family = Bytes.toBytes(familyDir.getName());
+
       Set<String> familyFiles = getTableRegionFamilyFiles(familyDir);
       List<SnapshotRegionManifest.StoreFile> snapshotFamilyFiles =
           snapshotFiles.remove(familyDir.getName());
+      List<StoreFileInfo> filesToTrack = new ArrayList<>();
       if (snapshotFamilyFiles != null) {
         List<SnapshotRegionManifest.StoreFile> hfilesToAdd = new ArrayList<>();
         for (SnapshotRegionManifest.StoreFile storeFile: snapshotFamilyFiles) {
           if (familyFiles.contains(storeFile.getName())) {
             // HFile already present
             familyFiles.remove(storeFile.getName());
+            //no need to restore already present files, but we need to add those to tracker
+            filesToTrack.add(new StoreFileInfo(conf, fs,
+              new Path(familyDir, storeFile.getName()), true));
           } else {
             // HFile missing
             hfilesToAdd.add(storeFile);
@@ -521,9 +536,11 @@ public class RestoreSnapshotHelper {
         // Remove hfiles not present in the snapshot
         for (String hfileName: familyFiles) {
           Path hfile = new Path(familyDir, hfileName);
-          LOG.trace("Removing HFile=" + hfileName + " not present in snapshot=" + snapshotName+
-            " from region=" + regionInfo.getEncodedName() + " table=" + tableName);
-          HFileArchiver.archiveStoreFile(conf, fs, regionInfo, tableDir, family, hfile);
+          if (!fs.getFileStatus(hfile).isDirectory()) {
+            LOG.trace("Removing HFile=" + hfileName + " not present in snapshot=" +
+              snapshotName + " from region=" + regionInfo.getEncodedName() + " table=" + tableName);
+            HFileArchiver.archiveStoreFile(conf, fs, regionInfo, tableDir, family, hfile);
+          }
         }
 
         // Restore Missing files
@@ -531,7 +548,10 @@ public class RestoreSnapshotHelper {
           LOG.debug("Restoring missing HFileLink " + storeFile.getName() +
                   " of snapshot=" + snapshotName+
                   " to region=" + regionInfo.getEncodedName() + " table=" + tableName);
-          restoreStoreFile(familyDir, regionInfo, storeFile, createBackRefs);
+          String fileName = restoreStoreFile(familyDir, regionInfo, storeFile, createBackRefs);
+          //mark the reference file to be added to tracker
+          filesToTrack.add(new StoreFileInfo(conf, fs,
+            new Path(familyDir, fileName), true));
         }
       } else {
         // Family doesn't exists in the snapshot
@@ -540,12 +560,24 @@ public class RestoreSnapshotHelper {
         HFileArchiver.archiveFamilyByFamilyDir(fs, conf, regionInfo, familyDir, family);
         fs.delete(familyDir, true);
       }
+
+      StoreFileTracker tracker = StoreFileTrackerFactory.create(conf, true,
+          StoreContext.getBuilder().withFamilyStoreDirectoryPath(familyDir).
+            withRegionFileSystem(regionFS).build());
+
+      //simply reset list of tracked files with the matching files
+      //and the extra one present in the snapshot
+      tracker.set(filesToTrack);
     }
 
     // Add families not present in the table
     for (Map.Entry<String, List<SnapshotRegionManifest.StoreFile>> familyEntry:
                                                                       snapshotFiles.entrySet()) {
       Path familyDir = new Path(regionDir, familyEntry.getKey());
+      StoreFileTracker tracker = StoreFileTrackerFactory.create(conf, true,
+          StoreContext.getBuilder().withFamilyStoreDirectoryPath(familyDir).
+            withRegionFileSystem(regionFS).build());
+      List<StoreFileInfo> files = new ArrayList<>();
       if (!fs.mkdirs(familyDir)) {
         throw new IOException("Unable to create familyDir=" + familyDir);
       }
@@ -553,8 +585,10 @@ public class RestoreSnapshotHelper {
       for (SnapshotRegionManifest.StoreFile storeFile: familyEntry.getValue()) {
         LOG.trace("Adding HFileLink (Not present in the table) " + storeFile.getName()
                 + " of snapshot " + snapshotName + " to table=" + tableName);
-        restoreStoreFile(familyDir, regionInfo, storeFile, createBackRefs);
+        String fileName = restoreStoreFile(familyDir, regionInfo, storeFile, createBackRefs);
+        files.add(new StoreFileInfo(conf, fs, new Path(familyDir, fileName), true));
       }
+      tracker.set(files);
     }
   }
 
@@ -627,7 +661,8 @@ public class RestoreSnapshotHelper {
       final RegionInfo region) throws IOException {
     // clone region info (change embedded tableName with the new one)
     Path clonedRegionPath = MobUtils.getMobRegionPath(rootDir, tableDesc.getTableName());
-    cloneRegion(clonedRegionPath, region, regionManifests.get(region.getEncodedName()));
+    cloneRegion(MobUtils.getMobRegionInfo(tableDesc.getTableName()),
+      clonedRegionPath, region, regionManifests.get(region.getEncodedName()));
   }
 
   /**
@@ -641,18 +676,44 @@ public class RestoreSnapshotHelper {
    * @param regionDir {@link Path} cloned dir
    * @param snapshotRegionInfo
    */
-  private void cloneRegion(final Path regionDir, final RegionInfo snapshotRegionInfo,
-      final SnapshotRegionManifest manifest) throws IOException {
+  private void cloneRegion(final RegionInfo newRegionInfo, final Path regionDir,
+      final RegionInfo snapshotRegionInfo, final SnapshotRegionManifest manifest)
+        throws IOException {
     final String tableName = tableDesc.getTableName().getNameAsString();
     final String snapshotName = snapshotDesc.getName();
     for (SnapshotRegionManifest.FamilyFiles familyFiles: manifest.getFamilyFilesList()) {
       Path familyDir = new Path(regionDir, familyFiles.getFamilyName().toStringUtf8());
+      List<StoreFileInfo> clonedFiles = new ArrayList<>();
       for (SnapshotRegionManifest.StoreFile storeFile: familyFiles.getStoreFilesList()) {
         LOG.info("Adding HFileLink " + storeFile.getName() +" from cloned region "
                 + "in snapshot " + snapshotName + " to table=" + tableName);
-        restoreStoreFile(familyDir, snapshotRegionInfo, storeFile, createBackRefs);
+        if (MobUtils.isMobRegionInfo(newRegionInfo)) {
+          String mobFileName = HFileLink.createHFileLinkName(snapshotRegionInfo,
+            storeFile.getName());
+          Path mobPath = new Path(familyDir, mobFileName);
+          if (fs.exists(mobPath)) {
+            fs.delete(mobPath, true);
+          }
+          restoreStoreFile(familyDir, snapshotRegionInfo, storeFile, createBackRefs);
+        } else {
+          String file = restoreStoreFile(familyDir, snapshotRegionInfo, storeFile, createBackRefs);
+          clonedFiles.add(new StoreFileInfo(conf, fs, new Path(familyDir, file), true));
+        }
+      }
+      //we don't need to track files under mobdir
+      if (!MobUtils.isMobRegionInfo(newRegionInfo)) {
+        Path regionPath = new Path(tableDir, newRegionInfo.getEncodedName());
+        HRegionFileSystem regionFS = (fs.exists(regionPath)) ?
+          HRegionFileSystem.openRegionFromFileSystem(conf, fs, tableDir, newRegionInfo, false) :
+          HRegionFileSystem.createRegionOnFileSystem(conf, fs, tableDir, newRegionInfo);
+
+        StoreFileTracker tracker = StoreFileTrackerFactory.create(conf, true,
+          StoreContext.getBuilder().withFamilyStoreDirectoryPath(familyDir).
+            withRegionFileSystem(regionFS).build());
+        tracker.set(clonedFiles);
       }
     }
+
   }
 
   /**
@@ -668,7 +729,9 @@ public class RestoreSnapshotHelper {
    */
   private void cloneRegion(final HRegion region, final RegionInfo snapshotRegionInfo,
       final SnapshotRegionManifest manifest) throws IOException {
-    cloneRegion(new Path(tableDir, region.getRegionInfo().getEncodedName()), snapshotRegionInfo,
+    cloneRegion(region.getRegionInfo(),
+      new Path(tableDir, region.getRegionInfo().getEncodedName()),
+      snapshotRegionInfo,
       manifest);
   }
 
@@ -685,16 +748,16 @@ public class RestoreSnapshotHelper {
    * @param createBackRef - Whether back reference should be created. Defaults to true.
    * @param storeFile store file name (can be a Reference, HFileLink or simple HFile)
    */
-  private void restoreStoreFile(final Path familyDir, final RegionInfo regionInfo,
+  private String restoreStoreFile(final Path familyDir, final RegionInfo regionInfo,
       final SnapshotRegionManifest.StoreFile storeFile, final boolean createBackRef)
           throws IOException {
     String hfileName = storeFile.getName();
     if (HFileLink.isHFileLink(hfileName)) {
-      HFileLink.createFromHFileLink(conf, fs, familyDir, hfileName, createBackRef);
+      return HFileLink.createFromHFileLink(conf, fs, familyDir, hfileName, createBackRef);
     } else if (StoreFileInfo.isReference(hfileName)) {
-      restoreReferenceFile(familyDir, regionInfo, storeFile);
+      return restoreReferenceFile(familyDir, regionInfo, storeFile);
     } else {
-      HFileLink.create(conf, fs, familyDir, regionInfo, hfileName, createBackRef);
+      return HFileLink.create(conf, fs, familyDir, regionInfo, hfileName, createBackRef);
     }
   }
 
@@ -716,7 +779,7 @@ public class RestoreSnapshotHelper {
    * @param regionInfo destination region info for the table
    * @param storeFile reference file name
    */
-  private void restoreReferenceFile(final Path familyDir, final RegionInfo regionInfo,
+  private String restoreReferenceFile(final Path familyDir, final RegionInfo regionInfo,
       final SnapshotRegionManifest.StoreFile storeFile) throws IOException {
     String hfileName = storeFile.getName();
 
@@ -760,6 +823,7 @@ public class RestoreSnapshotHelper {
       IOUtils.copyBytes(in, out, conf);
     }
 
+
     // Add the daughter region to the map
     String regionName = Bytes.toString(regionsMap.get(regionInfo.getEncodedNameAsBytes()));
     if (regionName == null) {
@@ -777,6 +841,7 @@ public class RestoreSnapshotHelper {
         daughters.setSecond(regionName);
       }
     }
+    return outPath.getName();
   }
 
   /**
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCloneSnapshotProcedureFileBasedSFT.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCloneSnapshotProcedureFileBasedSFT.java
new file mode 100644
index 0000000..f3ae128
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCloneSnapshotProcedureFileBasedSFT.java
@@ -0,0 +1,42 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.procedure;
+
+import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.TRACKER_IMPL;
+import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.Trackers.FILE;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.experimental.categories.Category;
+
+@Category({ MasterTests.class, MediumTests.class})
+public class TestCloneSnapshotProcedureFileBasedSFT extends TestCloneSnapshotProcedure {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestCloneSnapshotProcedureFileBasedSFT.class);
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    UTIL.getConfiguration().set(TRACKER_IMPL, FILE.name());
+    UTIL.getConfiguration().setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1);
+    UTIL.startMiniCluster(1);
+  }
+}

[hbase] 10/15: HBASE-26386 Refactor StoreFileTracker implementations to expose the set method (#3774)

Posted by zh...@apache.org.
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

commit ca07fb869fbe2e673bf2362fd1d8eba3b22f4d0e
Author: Duo Zhang <zh...@apache.org>
AuthorDate: Thu Oct 21 10:27:45 2021 +0800

    HBASE-26386 Refactor StoreFileTracker implementations to expose the set method (#3774)
    
    Signed-off-by: Wellington Chevreuil <wc...@apache.org>
---
 .../regionserver/storefiletracker/DefaultStoreFileTracker.java |  2 +-
 .../storefiletracker/FileBasedStoreFileTracker.java            |  2 +-
 .../storefiletracker/MigrationStoreFileTracker.java            |  5 +----
 .../hbase/regionserver/storefiletracker/StoreFileTracker.java  |  7 +++++--
 .../regionserver/storefiletracker/StoreFileTrackerBase.java    | 10 ----------
 5 files changed, 8 insertions(+), 18 deletions(-)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/DefaultStoreFileTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/DefaultStoreFileTracker.java
index a13b75b..b1e298d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/DefaultStoreFileTracker.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/DefaultStoreFileTracker.java
@@ -62,7 +62,7 @@ class DefaultStoreFileTracker extends StoreFileTrackerBase {
   }
 
   @Override
-  void set(List<StoreFileInfo> files) {
+  public void set(List<StoreFileInfo> files) {
     // NOOP
   }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/FileBasedStoreFileTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/FileBasedStoreFileTracker.java
index 4da7911..079b59b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/FileBasedStoreFileTracker.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/FileBasedStoreFileTracker.java
@@ -148,7 +148,7 @@ class FileBasedStoreFileTracker extends StoreFileTrackerBase {
   }
 
   @Override
-  void set(List<StoreFileInfo> files) throws IOException {
+  public void set(List<StoreFileInfo> files) throws IOException {
     synchronized (storefiles) {
       storefiles.clear();
       StoreFileList.Builder builder = StoreFileList.newBuilder();
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java
index 230c1ec..a6648f2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java
@@ -21,9 +21,6 @@ import java.io.IOException;
 import java.util.Collection;
 import java.util.List;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.client.TableDescriptor;
-import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
-import org.apache.hadoop.hbase.procedure2.util.StringUtils;
 import org.apache.hadoop.hbase.regionserver.StoreContext;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -84,7 +81,7 @@ class MigrationStoreFileTracker extends StoreFileTrackerBase {
   }
 
   @Override
-  void set(List<StoreFileInfo> files) {
+  public void set(List<StoreFileInfo> files) {
     throw new UnsupportedOperationException(
       "Should not call this method on " + getClass().getSimpleName());
   }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java
index fd8f7c9..f56a0dd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java
@@ -20,8 +20,6 @@ package org.apache.hadoop.hbase.regionserver.storefiletracker;
 import java.io.IOException;
 import java.util.Collection;
 import java.util.List;
-
-import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.regionserver.CreateStoreFileWriterParams;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
@@ -70,6 +68,11 @@ public interface StoreFileTracker {
     throws IOException;
 
   /**
+   * Set the store files.
+   */
+  void set(List<StoreFileInfo> files) throws IOException;
+
+  /**
    * Create a writer for writing new store files.
    * @return Writer for a new StoreFile
    */
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java
index edbaace..d3c5859 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java
@@ -21,11 +21,9 @@ import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTra
 
 import java.io.IOException;
 import java.util.Collection;
-import java.util.List;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
-import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.crypto.Encryption;
@@ -184,12 +182,4 @@ abstract class StoreFileTrackerBase implements StoreFileTracker {
 
   protected abstract void doAddCompactionResults(Collection<StoreFileInfo> compactedFiles,
     Collection<StoreFileInfo> newFiles) throws IOException;
-
-  /**
-   * used to mirror the store file list after loading when migration.
-   * <p/>
-   * Do not add this method to the {@link StoreFileTracker} interface since we do not need this
-   * method in upper layer.
-   */
-  abstract void set(List<StoreFileInfo> files) throws IOException;
 }

[hbase] 04/15: HBASE-26224 Introduce a MigrationStoreFileTracker to support migrating from different store file tracker implementations (#3656)

Posted by zh...@apache.org.
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

commit f298f9cacd34f80602ee8ffedc643c7c3425d081
Author: Duo Zhang <zh...@apache.org>
AuthorDate: Thu Sep 9 12:11:25 2021 +0800

    HBASE-26224 Introduce a MigrationStoreFileTracker to support migrating from different store file tracker implementations (#3656)
    
    Signed-off-by: Wellington Chevreuil <wc...@apache.org>
---
 .../storefiletracker/DefaultStoreFileTracker.java  |  10 +-
 .../FileBasedStoreFileTracker.java                 |  15 +-
 .../MigrationStoreFileTracker.java                 |  88 ++++++++++
 .../storefiletracker/StoreFileListFile.java        |   6 +-
 .../storefiletracker/StoreFileTrackerBase.java     |  12 +-
 .../storefiletracker/StoreFileTrackerFactory.java  |  40 +++--
 .../TestMigrationStoreFileTracker.java             | 193 +++++++++++++++++++++
 7 files changed, 343 insertions(+), 21 deletions(-)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/DefaultStoreFileTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/DefaultStoreFileTracker.java
index 22e0513..a13b75b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/DefaultStoreFileTracker.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/DefaultStoreFileTracker.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.regionserver.storefiletracker;
 
 import java.io.IOException;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
 import org.apache.hadoop.conf.Configuration;
 
@@ -39,7 +40,9 @@ class DefaultStoreFileTracker extends StoreFileTrackerBase {
 
   @Override
   public List<StoreFileInfo> load() throws IOException {
-    return ctx.getRegionFileSystem().getStoreFiles(ctx.getFamily().getNameAsString());
+    List<StoreFileInfo> files =
+      ctx.getRegionFileSystem().getStoreFiles(ctx.getFamily().getNameAsString());
+    return files != null ? files : Collections.emptyList();
   }
 
   @Override
@@ -57,4 +60,9 @@ class DefaultStoreFileTracker extends StoreFileTrackerBase {
     Collection<StoreFileInfo> newFiles) throws IOException {
     // NOOP
   }
+
+  @Override
+  void set(List<StoreFileInfo> files) {
+    // NOOP
+  }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/FileBasedStoreFileTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/FileBasedStoreFileTracker.java
index de28b0e..c370b87 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/FileBasedStoreFileTracker.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/FileBasedStoreFileTracker.java
@@ -48,7 +48,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.StoreFileTrackerProtos.
  * storages.
  */
 @InterfaceAudience.Private
-public class FileBasedStoreFileTracker extends StoreFileTrackerBase {
+class FileBasedStoreFileTracker extends StoreFileTrackerBase {
 
   private final StoreFileListFile backedFile;
 
@@ -139,4 +139,17 @@ public class FileBasedStoreFileTracker extends StoreFileTrackerBase {
       }
     }
   }
+
+  @Override
+  void set(List<StoreFileInfo> files) throws IOException {
+    synchronized (storefiles) {
+      storefiles.clear();
+      StoreFileList.Builder builder = StoreFileList.newBuilder();
+      for (StoreFileInfo info : files) {
+        storefiles.put(info.getPath().getName(), info);
+        builder.addStoreFile(toStoreFileEntry(info));
+      }
+      backedFile.update(builder);
+    }
+  }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java
new file mode 100644
index 0000000..e486e6d
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java
@@ -0,0 +1,88 @@
+/**
+ * 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.regionserver.storefiletracker;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.regionserver.StoreContext;
+import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
+
+/**
+ * A store file tracker used for migrating between store file tracker implementations.
+ */
+@InterfaceAudience.Private
+class MigrationStoreFileTracker extends StoreFileTrackerBase {
+
+  public static final String SRC_IMPL = "hbase.store.file-tracker.migration.src.impl";
+
+  public static final String DST_IMPL = "hbase.store.file-tracker.migration.dst.impl";
+
+  private final StoreFileTrackerBase src;
+
+  private final StoreFileTrackerBase dst;
+
+  public MigrationStoreFileTracker(Configuration conf, boolean isPrimaryReplica, StoreContext ctx) {
+    super(conf, isPrimaryReplica, ctx);
+    this.src = StoreFileTrackerFactory.create(conf, SRC_IMPL, isPrimaryReplica, ctx);
+    this.dst = StoreFileTrackerFactory.create(conf, DST_IMPL, isPrimaryReplica, ctx);
+    Preconditions.checkArgument(!src.getClass().equals(dst.getClass()),
+      "src and dst is the same: %s", src.getClass());
+  }
+
+  @Override
+  public List<StoreFileInfo> load() throws IOException {
+    List<StoreFileInfo> files = src.load();
+    dst.set(files);
+    return files;
+  }
+
+  @Override
+  protected boolean requireWritingToTmpDirFirst() {
+    // Returns true if either of the two StoreFileTracker returns true.
+    // For example, if we want to migrate from a tracker implementation which can ignore the broken
+    // files under data directory to a tracker implementation which can not, if we still allow
+    // writing in tmp directory directly, we may have some broken files under the data directory and
+    // then after we finally change the implementation which can not ignore the broken files, we
+    // will be in trouble.
+    return src.requireWritingToTmpDirFirst() || dst.requireWritingToTmpDirFirst();
+  }
+
+  @Override
+  protected void doAddNewStoreFiles(Collection<StoreFileInfo> newFiles) throws IOException {
+    src.doAddNewStoreFiles(newFiles);
+    dst.doAddNewStoreFiles(newFiles);
+  }
+
+  @Override
+  protected void doAddCompactionResults(Collection<StoreFileInfo> compactedFiles,
+    Collection<StoreFileInfo> newFiles) throws IOException {
+    src.doAddCompactionResults(compactedFiles, newFiles);
+    dst.doAddCompactionResults(compactedFiles, newFiles);
+  }
+
+  @Override
+  void set(List<StoreFileInfo> files) {
+    throw new UnsupportedOperationException(
+      "Should not call this method on " + getClass().getSimpleName());
+  }
+}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileListFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileListFile.java
index c778bfc..ffb3647 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileListFile.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileListFile.java
@@ -29,7 +29,6 @@ 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.hbase.thirdparty.com.google.common.io.ByteStreams;
 import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
 
@@ -121,7 +120,10 @@ class StoreFileListFile {
    * We will set the timestamp in this method so just pass the builder in
    */
   void update(StoreFileList.Builder builder) throws IOException {
-    Preconditions.checkState(nextTrackFile >= 0, "should call load first before calling update");
+    if (nextTrackFile < 0) {
+      // we need to call load first to load the prevTimestamp and also the next file
+      load();
+    }
     FileSystem fs = ctx.getRegionFileSystem().getFileSystem();
     long timestamp = Math.max(prevTimestamp + 1, EnvironmentEdgeManager.currentTime());
     try (FSDataOutputStream out = fs.create(trackFiles[nextTrackFile], true)) {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java
index 92c6992..d860f8e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.regionserver.storefiletracker;
 
 import java.io.IOException;
 import java.util.Collection;
+import java.util.List;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
@@ -95,8 +96,7 @@ abstract class StoreFileTrackerBase implements StoreFileTracker {
   }
 
   @Override
-  public final StoreFileWriter createWriter(CreateStoreFileWriterParams params)
-    throws IOException {
+  public final StoreFileWriter createWriter(CreateStoreFileWriterParams params) throws IOException {
     if (!isPrimaryReplica) {
       throw new IllegalStateException("Should not call create writer on secondary replicas");
     }
@@ -170,4 +170,12 @@ abstract class StoreFileTrackerBase implements StoreFileTracker {
 
   protected abstract void doAddCompactionResults(Collection<StoreFileInfo> compactedFiles,
     Collection<StoreFileInfo> newFiles) throws IOException;
+
+  /**
+   * used to mirror the store file list after loading when migration.
+   * <p/>
+   * Do not add this method to the {@link StoreFileTracker} interface since we do not need this
+   * method in upper layer.
+   */
+  abstract void set(List<StoreFileInfo> files) throws IOException;
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java
index c446d5a..6eb269e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java
@@ -18,18 +18,20 @@
 package org.apache.hadoop.hbase.regionserver.storefiletracker;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.CompoundConfiguration;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
 import org.apache.hadoop.hbase.regionserver.StoreContext;
+import org.apache.hadoop.hbase.regionserver.StoreUtils;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ReflectionUtils;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
+
 /**
  * Factory method for creating store file tracker.
  */
@@ -39,7 +41,7 @@ public final class StoreFileTrackerFactory {
   private static final Logger LOG = LoggerFactory.getLogger(StoreFileTrackerFactory.class);
 
   public static StoreFileTracker create(Configuration conf, boolean isPrimaryReplica,
-      StoreContext ctx) {
+    StoreContext ctx) {
     Class<? extends StoreFileTracker> tracker =
       conf.getClass(TRACK_IMPL, DefaultStoreFileTracker.class, StoreFileTracker.class);
     LOG.info("instantiating StoreFileTracker impl {}", tracker.getName());
@@ -47,22 +49,30 @@ public final class StoreFileTrackerFactory {
   }
 
   public static StoreFileTracker create(Configuration conf, boolean isPrimaryReplica, String family,
-      HRegionFileSystem regionFs) {
+    HRegionFileSystem regionFs) {
     ColumnFamilyDescriptorBuilder fDescBuilder =
       ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(family));
-    StoreContext ctx = StoreContext.getBuilder().
-      withColumnFamilyDescriptor(fDescBuilder.build()).
-      withRegionFileSystem(regionFs).
-      build();
-    return StoreFileTrackerFactory.create(conf, isPrimaryReplica, ctx);
+    StoreContext ctx = StoreContext.getBuilder().withColumnFamilyDescriptor(fDescBuilder.build())
+      .withRegionFileSystem(regionFs).build();
+    return StoreFileTrackerFactory.create(conf, TRACK_IMPL, isPrimaryReplica, ctx);
+  }
+
+  public static Configuration mergeConfigurations(Configuration global, TableDescriptor table,
+    ColumnFamilyDescriptor family) {
+    return StoreUtils.createStoreConfiguration(global, table, family);
   }
 
-  public static Configuration mergeConfigurations(Configuration global,
-    TableDescriptor table, ColumnFamilyDescriptor family) {
-    return new CompoundConfiguration()
-      .add(global)
-      .addBytesMap(table.getValues())
-      .addStringMap(family.getConfiguration())
-      .addBytesMap(family.getValues());
+  static StoreFileTrackerBase create(Configuration conf, String configName,
+    boolean isPrimaryReplica, StoreContext ctx) {
+    String className =
+      Preconditions.checkNotNull(conf.get(configName), "config %s is not set", configName);
+    Class<? extends StoreFileTrackerBase> tracker;
+    try {
+      tracker = Class.forName(className).asSubclass(StoreFileTrackerBase.class);
+    } catch (ClassNotFoundException e) {
+      throw new RuntimeException(e);
+    }
+    LOG.info("instantiating StoreFileTracker impl {} as {}", tracker.getName(), configName);
+    return ReflectionUtils.newInstance(tracker, conf, isPrimaryReplica, ctx);
   }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestMigrationStoreFileTracker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestMigrationStoreFileTracker.java
new file mode 100644
index 0000000..e9ebb4c
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestMigrationStoreFileTracker.java
@@ -0,0 +1,193 @@
+/**
+ * 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.regionserver.storefiletracker;
+
+import static org.hamcrest.CoreMatchers.hasItems;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtil;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.regionserver.ChunkCreator;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.MemStoreLAB;
+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.wal.WAL;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameter;
+import org.junit.runners.Parameterized.Parameters;
+
+import org.apache.hbase.thirdparty.com.google.common.collect.Iterables;
+import org.apache.hbase.thirdparty.com.google.common.io.Closeables;
+
+@RunWith(Parameterized.class)
+@Category({ RegionServerTests.class, MediumTests.class })
+public class TestMigrationStoreFileTracker {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestMigrationStoreFileTracker.class);
+
+  private static final HBaseTestingUtil UTIL = new HBaseTestingUtil();
+
+  private static final byte[] CF = Bytes.toBytes("cf");
+
+  private static final byte[] CQ = Bytes.toBytes("cq");
+
+  private static final TableDescriptor TD =
+    TableDescriptorBuilder.newBuilder(TableName.valueOf("file_based_tracker"))
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(CF)).build();
+
+  private static final RegionInfo RI = RegionInfoBuilder.newBuilder(TD.getTableName()).build();
+
+  @Rule
+  public TestName name = new TestName();
+
+  @Parameter(0)
+  public Class<? extends StoreFileTrackerBase> srcImplClass;
+
+  @Parameter(1)
+  public Class<? extends StoreFileTrackerBase> dstImplClass;
+
+  private HRegion region;
+
+  private Path rootDir;
+
+  private WAL wal;
+
+  @Parameters(name = "{index}: src={0}, dst={1}")
+  public static List<Object[]> params() {
+    List<Class<? extends StoreFileTrackerBase>> impls =
+      Arrays.asList(DefaultStoreFileTracker.class, FileBasedStoreFileTracker.class);
+    List<Object[]> params = new ArrayList<>();
+    for (Class<? extends StoreFileTrackerBase> src : impls) {
+      for (Class<? extends StoreFileTrackerBase> dst : impls) {
+        if (src.equals(dst)) {
+          continue;
+        }
+        params.add(new Object[] { src, dst });
+      }
+    }
+    return params;
+  }
+
+  @BeforeClass
+  public static void setUpBeforeClass() {
+    ChunkCreator.initialize(MemStoreLAB.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null,
+      MemStoreLAB.INDEX_CHUNK_SIZE_PERCENTAGE_DEFAULT);
+  }
+
+  @Before
+  public void setUp() throws IOException {
+    Configuration conf = UTIL.getConfiguration();
+    conf.setClass(MigrationStoreFileTracker.SRC_IMPL, srcImplClass, StoreFileTrackerBase.class);
+    conf.setClass(MigrationStoreFileTracker.DST_IMPL, dstImplClass, StoreFileTrackerBase.class);
+    rootDir = UTIL.getDataTestDir(name.getMethodName().replaceAll("[=:\\[ ]", "_"));
+    wal = HBaseTestingUtil.createWal(conf, rootDir, RI);
+  }
+
+  @After
+  public void tearDown() throws IOException {
+    if (region != null) {
+      region.close();
+    }
+    Closeables.close(wal, true);
+    UTIL.cleanupTestDir();
+  }
+
+  private List<String> getStoreFiles() {
+    return Iterables.getOnlyElement(region.getStores()).getStorefiles().stream()
+      .map(s -> s.getFileInfo().getPath().getName()).collect(Collectors.toList());
+  }
+
+  private HRegion createRegion(Class<? extends StoreFileTrackerBase> trackerImplClass)
+    throws IOException {
+    Configuration conf = new Configuration(UTIL.getConfiguration());
+    conf.setClass(StoreFileTrackerFactory.TRACK_IMPL, trackerImplClass, StoreFileTracker.class);
+    return HRegion.createHRegion(RI, rootDir, conf, TD, wal, true);
+  }
+
+  private void reopenRegion(Class<? extends StoreFileTrackerBase> trackerImplClass)
+    throws IOException {
+    region.flush(true);
+    List<String> before = getStoreFiles();
+    region.close();
+    Configuration conf = new Configuration(UTIL.getConfiguration());
+    conf.setClass(StoreFileTrackerFactory.TRACK_IMPL, trackerImplClass, StoreFileTracker.class);
+    region = HRegion.openHRegion(rootDir, RI, TD, wal, conf);
+    List<String> after = getStoreFiles();
+    assertEquals(before.size(), after.size());
+    assertThat(after, hasItems(before.toArray(new String[0])));
+  }
+
+  private void putData(int start, int end) throws IOException {
+    for (int i = start; i < end; i++) {
+      region.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i)));
+      if (i % 30 == 0) {
+        region.flush(true);
+      }
+    }
+  }
+
+  private void verifyData(int start, int end) throws IOException {
+    for (int i = start; i < end; i++) {
+      Result result = region.get(new Get(Bytes.toBytes(i)));
+      assertEquals(i, Bytes.toInt(result.getValue(CF, CQ)));
+    }
+  }
+
+  @Test
+  public void testMigration() throws IOException {
+    region = createRegion(srcImplClass);
+    putData(0, 100);
+    verifyData(0, 100);
+    reopenRegion(MigrationStoreFileTracker.class);
+    verifyData(0, 100);
+    region.compact(true);
+    putData(100, 200);
+    reopenRegion(dstImplClass);
+    verifyData(0, 200);
+  }
+}

[hbase] 09/15: HBASE-26326 CreateTableProcedure fails when FileBasedStoreFileTracker… (#3721)

Posted by zh...@apache.org.
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

commit 030a2a83cacf7d19078760c8bbe4d9dbbe21a0d0
Author: Wellington Ramos Chevreuil <wc...@apache.org>
AuthorDate: Wed Oct 13 15:48:13 2021 +0100

    HBASE-26326 CreateTableProcedure fails when FileBasedStoreFileTracker… (#3721)
    
    Signed-off-by: Duo Zhang <zh...@apache.org>
    Signed-off-by: Josh Elser <el...@apache.org>
---
 .../hbase/master/procedure/CreateTableProcedure.java |  6 ++----
 .../storefiletracker/FileBasedStoreFileTracker.java  |  9 ++++++++-
 .../storefiletracker/MigrationStoreFileTracker.java  | 12 +-----------
 .../storefiletracker/StoreFileTracker.java           |  5 +++--
 .../storefiletracker/StoreFileTrackerBase.java       |  9 ++++-----
 .../storefiletracker/StoreFileTrackerFactory.java    | 20 ++++++++++++++------
 .../master/procedure/TestCreateTableProcedure.java   | 16 ++++++++++++++++
 .../storefiletracker/TestStoreFileTracker.java       |  2 +-
 8 files changed, 49 insertions(+), 30 deletions(-)

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 80ed96a..55e3212 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
@@ -32,7 +32,6 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
 import org.apache.hadoop.hbase.client.TableDescriptor;
-import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.client.TableState;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.MasterFileSystem;
@@ -270,9 +269,8 @@ public class CreateTableProcedure
         (newRegions != null ? newRegions.size() : 0));
     }
 
-    TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableDescriptor);
-    StoreFileTrackerFactory.persistTrackerConfig(env.getMasterConfiguration(), builder);
-    tableDescriptor = builder.build();
+    tableDescriptor = StoreFileTrackerFactory.updateWithTrackerConfigs(env.getMasterConfiguration(),
+      tableDescriptor);
 
     final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
     if (cpHost != null) {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/FileBasedStoreFileTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/FileBasedStoreFileTracker.java
index c370b87..4da7911 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/FileBasedStoreFileTracker.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/FileBasedStoreFileTracker.java
@@ -56,7 +56,14 @@ class FileBasedStoreFileTracker extends StoreFileTrackerBase {
 
   public FileBasedStoreFileTracker(Configuration conf, boolean isPrimaryReplica, StoreContext ctx) {
     super(conf, isPrimaryReplica, ctx);
-    backedFile = new StoreFileListFile(ctx);
+    //CreateTableProcedure needs to instantiate the configured SFT impl, in order to update table
+    //descriptors with the SFT impl specific configs. By the time this happens, the table has no
+    //regions nor stores yet, so it can't create a proper StoreContext.
+    if (ctx != null) {
+      backedFile = new StoreFileListFile(ctx);
+    } else {
+      backedFile = null;
+    }
   }
 
   @Override
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java
index 1946d4b..230c1ec 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/MigrationStoreFileTracker.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.util.Collection;
 import java.util.List;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.procedure2.util.StringUtils;
 import org.apache.hadoop.hbase.regionserver.StoreContext;
@@ -88,17 +89,6 @@ class MigrationStoreFileTracker extends StoreFileTrackerBase {
       "Should not call this method on " + getClass().getSimpleName());
   }
 
-  @Override
-  public void persistConfiguration(TableDescriptorBuilder builder) {
-    super.persistConfiguration(builder);
-    if (StringUtils.isEmpty(builder.getValue(SRC_IMPL))) {
-      builder.setValue(SRC_IMPL, src.getTrackerName());
-    }
-    if (StringUtils.isEmpty(builder.getValue(DST_IMPL))) {
-      builder.setValue(DST_IMPL, dst.getTrackerName());
-    }
-  }
-
   static Class<? extends StoreFileTracker> getSrcTrackerClass(Configuration conf) {
     return StoreFileTrackerFactory.getStoreFileTrackerClassForMigration(conf, SRC_IMPL);
   }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java
index 59fe7ef..fd8f7c9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTracker.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.util.Collection;
 import java.util.List;
 
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.regionserver.CreateStoreFileWriterParams;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
@@ -75,7 +76,7 @@ public interface StoreFileTracker {
   StoreFileWriter createWriter(CreateStoreFileWriterParams params) throws IOException;
 
   /**
-   * Saves StoreFileTracker implementations specific configurations into the table descriptors.
+   * Adds StoreFileTracker implementations specific configurations into the table descriptor.
    * <p/>
    * This is used to avoid accidentally data loss when changing the cluster level store file tracker
    * implementation, and also possible misconfiguration between master and region servers.
@@ -83,5 +84,5 @@ public interface StoreFileTracker {
    * See HBASE-26246 for more details.
    * @param builder The table descriptor builder for the given table.
    */
-  void persistConfiguration(TableDescriptorBuilder builder);
+  TableDescriptorBuilder updateWithTrackerConfigs(TableDescriptorBuilder builder);
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java
index a786add..edbaace 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java
@@ -25,6 +25,7 @@ import java.util.List;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.crypto.Encryption;
@@ -32,7 +33,6 @@ import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.io.hfile.HFileContext;
 import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
-import org.apache.hadoop.hbase.procedure2.util.StringUtils;
 import org.apache.hadoop.hbase.regionserver.CreateStoreFileWriterParams;
 import org.apache.hadoop.hbase.regionserver.StoreContext;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
@@ -83,10 +83,9 @@ abstract class StoreFileTrackerBase implements StoreFileTracker {
   }
 
   @Override
-  public void persistConfiguration(TableDescriptorBuilder builder) {
-    if (StringUtils.isEmpty(builder.getValue(TRACKER_IMPL))) {
-      builder.setValue(TRACKER_IMPL, getTrackerName());
-    }
+  public TableDescriptorBuilder updateWithTrackerConfigs(TableDescriptorBuilder builder) {
+    builder.setValue(TRACKER_IMPL, getTrackerName());
+    return builder;
   }
 
   protected final String getTrackerName() {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java
index b586027..1c683ae 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerFactory.java
@@ -24,8 +24,10 @@ import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.procedure2.util.StringUtils;
 import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
 import org.apache.hadoop.hbase.regionserver.StoreContext;
+
 import org.apache.hadoop.hbase.regionserver.StoreUtils;
 import org.apache.hadoop.hbase.util.ReflectionUtils;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -158,12 +160,18 @@ public final class StoreFileTrackerFactory {
     return ReflectionUtils.newInstance(tracker, conf, isPrimaryReplica, ctx);
   }
 
-  public static void persistTrackerConfig(Configuration conf, TableDescriptorBuilder builder) {
-    TableDescriptor tableDescriptor = builder.build();
-    ColumnFamilyDescriptor cfDesc = tableDescriptor.getColumnFamilies()[0];
-    StoreContext context = StoreContext.getBuilder().withColumnFamilyDescriptor(cfDesc).build();
-    StoreFileTracker tracker = StoreFileTrackerFactory.create(conf, true, context);
-    tracker.persistConfiguration(builder);
+  public static TableDescriptor updateWithTrackerConfigs(Configuration conf,
+      TableDescriptor descriptor) {
+    //CreateTableProcedure needs to instantiate the configured SFT impl, in order to update table
+    //descriptors with the SFT impl specific configs. By the time this happens, the table has no
+    //regions nor stores yet, so it can't create a proper StoreContext.
+    if (StringUtils.isEmpty(descriptor.getValue(TRACKER_IMPL))) {
+      StoreFileTracker tracker =
+        StoreFileTrackerFactory.create(conf, true, null);
+      TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(descriptor);
+      return tracker.updateWithTrackerConfigs(builder).build();
+    }
+    return descriptor;
   }
 
   // should not use MigrationStoreFileTracker for new family
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 f432c80..51ea9f5 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
@@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.master.MasterFileSystem;
 import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
+import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory;
 import org.apache.hadoop.hbase.regionserver.storefiletracker.TestStoreFileTracker;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -106,6 +107,21 @@ public class TestCreateTableProcedure extends TestTableDDLProcedureBase {
   }
 
   @Test
+  public void testCreateWithFileBasedStoreTrackerImpl() throws Exception {
+    ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+    procExec.getEnvironment().getMasterConfiguration().set(StoreFileTrackerFactory.TRACKER_IMPL,
+      StoreFileTrackerFactory.Trackers.FILE.name());
+    final TableName tableName = TableName.valueOf(name.getMethodName());
+    TableDescriptor htd = MasterProcedureTestingUtility.createHTD(tableName, F1);
+    RegionInfo[] regions = ModifyRegionUtils.createRegionInfos(htd, null);
+    long procId = ProcedureTestingUtility.submitAndWait(procExec,
+      new CreateTableProcedure(procExec.getEnvironment(), htd, regions));
+    ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId));
+    htd = getMaster().getTableDescriptors().get(tableName);
+    assertEquals(StoreFileTrackerFactory.Trackers.FILE.name(), htd.getValue(TRACKER_IMPL));
+  }
+
+  @Test
   public void testCreateWithoutColumnFamily() throws Exception {
     final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
     final TableName tableName = TableName.valueOf(name.getMethodName());
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTracker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTracker.java
index 1dc9c4e..b30ca47 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTracker.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTracker.java
@@ -40,7 +40,7 @@ public class TestStoreFileTracker extends DefaultStoreFileTracker {
 
   public TestStoreFileTracker(Configuration conf, boolean isPrimaryReplica, StoreContext ctx) {
     super(conf, isPrimaryReplica, ctx);
-    if (ctx.getRegionFileSystem() != null) {
+    if (ctx != null && ctx.getRegionFileSystem() != null) {
       this.storeId = ctx.getRegionInfo().getEncodedName() + "-" + ctx.getFamily().getNameAsString();
       LOG.info("created storeId: {}", storeId);
       trackedFiles.computeIfAbsent(storeId, v -> new ArrayList<>());