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/28 13:08:24 UTC

[hbase] branch master updated: HBASE-26587 Introduce a new Admin API to change SFT implementation (#4030)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new f278a4c  HBASE-26587 Introduce a new Admin API to change SFT implementation (#4030)
f278a4c is described below

commit f278a4c98dc954bb77f52fdde7de42105ad7bf4f
Author: Duo Zhang <zh...@apache.org>
AuthorDate: Fri Jan 28 21:07:55 2022 +0800

    HBASE-26587 Introduce a new Admin API to change SFT implementation (#4030)
    
    Signed-off-by: Wellington Ramos Chevreuil <wc...@apache.org>
    Reviewed-by: Josh Elser <el...@apache.org>
---
 .../java/org/apache/hadoop/hbase/client/Admin.java |  48 +++-
 .../hadoop/hbase/client/AdminOverAsyncAdmin.java   |  14 +-
 .../org/apache/hadoop/hbase/client/AsyncAdmin.java |  16 ++
 .../hadoop/hbase/client/AsyncHBaseAdmin.java       |  11 +
 .../hadoop/hbase/client/RawAsyncHBaseAdmin.java    |  59 ++++-
 .../hbase/shaded/protobuf/RequestConverter.java    |  27 ++
 .../src/main/protobuf/server/master/Master.proto   |  29 +++
 .../protobuf/server/master/MasterProcedure.proto   |  15 ++
 .../hadoop/hbase/coprocessor/MasterObserver.java   |  52 ++++
 .../org/apache/hadoop/hbase/master/HMaster.java    |  54 ++++
 .../hadoop/hbase/master/MasterCoprocessorHost.java |  49 ++++
 .../hadoop/hbase/master/MasterRpcServices.java     |  30 +++
 .../apache/hadoop/hbase/master/MasterServices.java |  19 ++
 .../master/procedure/CreateTableProcedure.java     |   4 +-
 .../master/procedure/ModifyTableProcedure.java     |   4 +-
 .../hbase/master/snapshot/SnapshotManager.java     |   6 +-
 ...odifyColumnFamilyStoreFileTrackerProcedure.java | 107 ++++++++
 .../ModifyStoreFileTrackerProcedure.java           | 271 +++++++++++++++++++++
 .../ModifyTableStoreFileTrackerProcedure.java      |  71 ++++++
 .../storefiletracker/StoreFileTrackerFactory.java  | 192 ++-------------
 .../StoreFileTrackerValidationUtils.java           | 212 ++++++++++++++++
 .../hbase/security/access/AccessController.java    |  17 ++
 .../org/apache/hadoop/hbase/client/TestAdmin3.java | 157 ++++++++++++
 .../TestAsyncAdminModifyStoreFileTracker.java      | 197 +++++++++++++++
 .../hbase/master/MockNoopMasterServices.java       |  12 +
 .../TestStoreFileTrackerFactory.java               |  52 ----
 ...va => TestStoreFileTrackerValidationUtils.java} |  40 +--
 .../hbase/rsgroup/VerifyingRSGroupAdmin.java       |  26 +-
 .../hadoop/hbase/thrift2/client/ThriftAdmin.java   |  14 ++
 29 files changed, 1522 insertions(+), 283 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 6c36660..5d4337e 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
@@ -69,7 +69,6 @@ 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
@@ -501,6 +500,31 @@ public interface Admin extends Abortable, Closeable {
       throws IOException;
 
   /**
+   * Change the store file tracker of the given table's given family.
+   * @param tableName the table you want to change
+   * @param family the family you want to change
+   * @param dstSFT the destination store file tracker
+   * @throws IOException if a remote or network exception occurs
+   */
+  default void modifyColumnFamilyStoreFileTracker(TableName tableName, byte[] family, String dstSFT)
+    throws IOException {
+    get(modifyColumnFamilyStoreFileTrackerAsync(tableName, family, dstSFT), getSyncWaitTimeout(),
+      TimeUnit.MILLISECONDS);
+  }
+
+  /**
+   * Change the store file tracker of the given table's given family.
+   * @param tableName the table you want to change
+   * @param family the family you want to change
+   * @param dstSFT the destination store file tracker
+   * @return the result of the async modify. You can use Future.get(long, TimeUnit) to wait on the
+   *         operation to complete
+   * @throws IOException if a remote or network exception occurs
+   */
+  Future<Void> modifyColumnFamilyStoreFileTrackerAsync(TableName tableName, byte[] family,
+    String dstSFT) throws IOException;
+
+  /**
    * Get all the online regions on a region server.
    *
    * @return List of {@link RegionInfo}
@@ -1059,6 +1083,28 @@ public interface Admin extends Abortable, Closeable {
   Future<Void> modifyTableAsync(TableDescriptor td) throws IOException;
 
   /**
+   * Change the store file tracker of the given table.
+   * @param tableName the table you want to change
+   * @param dstSFT the destination store file tracker
+   * @throws IOException if a remote or network exception occurs
+   */
+  default void modifyTableStoreFileTracker(TableName tableName, String dstSFT) throws IOException {
+    get(modifyTableStoreFileTrackerAsync(tableName, dstSFT), getSyncWaitTimeout(),
+      TimeUnit.MILLISECONDS);
+  }
+
+  /**
+   * Change the store file tracker of the given table.
+   * @param tableName the table you want to change
+   * @param dstSFT the destination store file tracker
+   * @return the result of the async modify. You can use Future.get(long, TimeUnit) to wait on the
+   *         operation to complete
+   * @throws IOException if a remote or network exception occurs
+   */
+  Future<Void> modifyTableStoreFileTrackerAsync(TableName tableName, String dstSFT)
+    throws IOException;
+
+  /**
    * Shuts down the HBase cluster.
    * <p/>
    * Notice that, a success shutdown call may ends with an error since the remote server has already
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AdminOverAsyncAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AdminOverAsyncAdmin.java
index 161fe3d..4559e90 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AdminOverAsyncAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AdminOverAsyncAdmin.java
@@ -230,11 +230,17 @@ class AdminOverAsyncAdmin implements Admin {
 
   @Override
   public Future<Void> modifyColumnFamilyAsync(TableName tableName,
-      ColumnFamilyDescriptor columnFamily) throws IOException {
+    ColumnFamilyDescriptor columnFamily) throws IOException {
     return admin.modifyColumnFamily(tableName, columnFamily);
   }
 
   @Override
+  public Future<Void> modifyColumnFamilyStoreFileTrackerAsync(TableName tableName, byte[] family,
+    String dstSFT) throws IOException {
+    return admin.modifyColumnFamilyStoreFileTracker(tableName, family, dstSFT);
+  }
+
+  @Override
   public List<RegionInfo> getRegions(ServerName serverName) throws IOException {
     return get(admin.getRegions(serverName));
   }
@@ -476,6 +482,12 @@ class AdminOverAsyncAdmin implements Admin {
   }
 
   @Override
+  public Future<Void> modifyTableStoreFileTrackerAsync(TableName tableName, String dstSFT)
+    throws IOException {
+    return admin.modifyTableStoreFileTracker(tableName, dstSFT);
+  }
+
+  @Override
   public void shutdown() throws IOException {
     get(admin.shutdown());
   }
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 ba54bb73..b0dc0c1 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
@@ -184,6 +184,13 @@ public interface AsyncAdmin {
   CompletableFuture<Void> modifyTable(TableDescriptor desc);
 
   /**
+   * Change the store file tracker of the given table.
+   * @param tableName the table you want to change
+   * @param dstSFT the destination store file tracker
+   */
+  CompletableFuture<Void> modifyTableStoreFileTracker(TableName tableName, String dstSFT);
+
+  /**
    * Deletes a table.
    * @param tableName name of table to delete
    */
@@ -253,6 +260,15 @@ public interface AsyncAdmin {
       ColumnFamilyDescriptor columnFamily);
 
   /**
+   * Change the store file tracker of the given table's given family.
+   * @param tableName the table you want to change
+   * @param family the family you want to change
+   * @param dstSFT the destination store file tracker
+   */
+  CompletableFuture<Void> modifyColumnFamilyStoreFileTracker(TableName tableName, byte[] family,
+    String dstSFT);
+
+  /**
    * Create a new namespace.
    * @param descriptor descriptor which describes the new namespace
    */
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 64a13c7..fc7ee5c 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
@@ -147,6 +147,11 @@ class AsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
+  public CompletableFuture<Void> modifyTableStoreFileTracker(TableName tableName, String dstSFT) {
+    return wrap(rawAdmin.modifyTableStoreFileTracker(tableName, dstSFT));
+  }
+
+  @Override
   public CompletableFuture<Void> deleteTable(TableName tableName) {
     return wrap(rawAdmin.deleteTable(tableName));
   }
@@ -199,6 +204,12 @@ class AsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
+  public CompletableFuture<Void> modifyColumnFamilyStoreFileTracker(TableName tableName,
+    byte[] family, String dstSFT) {
+    return wrap(rawAdmin.modifyColumnFamilyStoreFileTracker(tableName, family, dstSFT));
+  }
+
+  @Override
   public CompletableFuture<Void> createNamespace(NamespaceDescriptor descriptor) {
     return wrap(rawAdmin.createNamespace(descriptor));
   }
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 f0895a0..b7bf6c5 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
@@ -21,6 +21,7 @@ import static org.apache.hadoop.hbase.HConstants.HIGH_QOS;
 import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME;
 import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
 import static org.apache.hadoop.hbase.util.FutureUtils.unwrapCompletionException;
+
 import edu.umd.cs.findbugs.annotations.Nullable;
 import java.io.IOException;
 import java.util.ArrayList;
@@ -230,10 +231,14 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTable
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnStoreFileTrackerRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnStoreFileTrackerResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableStoreFileTrackerRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableStoreFileTrackerResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest;
@@ -661,6 +666,18 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
+  public CompletableFuture<Void> modifyTableStoreFileTracker(TableName tableName, String dstSFT) {
+    return this
+      .<ModifyTableStoreFileTrackerRequest, ModifyTableStoreFileTrackerResponse> procedureCall(
+        tableName,
+        RequestConverter.buildModifyTableStoreFileTrackerRequest(tableName, dstSFT,
+          ng.getNonceGroup(), ng.newNonce()),
+        (s, c, req, done) -> s.modifyTableStoreFileTracker(c, req, done),
+        (resp) -> resp.getProcId(),
+        new ModifyTableStoreFileTrackerProcedureBiConsumer(this, tableName));
+  }
+
+  @Override
   public CompletableFuture<Void> deleteTable(TableName tableName) {
     return this.<DeleteTableRequest, DeleteTableResponse> procedureCall(tableName,
       RequestConverter.buildDeleteTableRequest(tableName, ng.getNonceGroup(), ng.newNonce()),
@@ -809,6 +826,19 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
+  public CompletableFuture<Void> modifyColumnFamilyStoreFileTracker(TableName tableName,
+    byte[] family, String dstSFT) {
+    return this
+      .<ModifyColumnStoreFileTrackerRequest, ModifyColumnStoreFileTrackerResponse> procedureCall(
+        tableName,
+        RequestConverter.buildModifyColumnStoreFileTrackerRequest(tableName, family, dstSFT,
+          ng.getNonceGroup(), ng.newNonce()),
+        (s, c, req, done) -> s.modifyColumnStoreFileTracker(c, req, done),
+        (resp) -> resp.getProcId(),
+        new ModifyColumnFamilyStoreFileTrackerProcedureBiConsumer(tableName));
+  }
+
+  @Override
   public CompletableFuture<Void> createNamespace(NamespaceDescriptor descriptor) {
     return this.<CreateNamespaceRequest, CreateNamespaceResponse> procedureCall(
       RequestConverter.buildCreateNamespaceRequest(descriptor),
@@ -2590,7 +2620,20 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
 
     @Override
     String getOperationType() {
-      return "ENABLE";
+      return "MODIFY";
+    }
+  }
+
+  private static class ModifyTableStoreFileTrackerProcedureBiConsumer
+    extends TableProcedureBiConsumer {
+
+    ModifyTableStoreFileTrackerProcedureBiConsumer(AsyncAdmin admin, TableName tableName) {
+      super(tableName);
+    }
+
+    @Override
+    String getOperationType() {
+      return "MODIFY_TABLE_STORE_FILE_TRACKER";
     }
   }
 
@@ -2684,6 +2727,19 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
     }
   }
 
+  private static class ModifyColumnFamilyStoreFileTrackerProcedureBiConsumer
+    extends TableProcedureBiConsumer {
+
+    ModifyColumnFamilyStoreFileTrackerProcedureBiConsumer(TableName tableName) {
+      super(tableName);
+    }
+
+    @Override
+    String getOperationType() {
+      return "MODIFY_COLUMN_FAMILY_STORE_FILE_TRACKER";
+    }
+  }
+
   private static class CreateNamespaceProcedureBiConsumer extends NamespaceProcedureBiConsumer {
 
     CreateNamespaceProcedureBiConsumer(String namespaceName) {
@@ -4282,5 +4338,4 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
         return CompletableFuture.completedFuture(Collections.emptyList());
     }
   }
-
 }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
index 821f731..3008956 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
@@ -69,6 +69,7 @@ import org.apache.hadoop.security.token.Token;
 import org.apache.yetus.audience.InterfaceAudience;
 
 import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
+import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
 import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
 import org.apache.hbase.thirdparty.org.apache.commons.collections4.MapUtils;
 
@@ -128,8 +129,10 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshot
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnStoreFileTrackerRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableStoreFileTrackerRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionRequest;
@@ -1022,6 +1025,19 @@ public final class RequestConverter {
     return builder.build();
   }
 
+  public static ModifyColumnStoreFileTrackerRequest
+    buildModifyColumnStoreFileTrackerRequest(final TableName tableName, final byte[] family,
+      final String dstSFT, final long nonceGroup, final long nonce) {
+    ModifyColumnStoreFileTrackerRequest.Builder builder =
+      ModifyColumnStoreFileTrackerRequest.newBuilder();
+    builder.setTableName(ProtobufUtil.toProtoTableName((tableName)));
+    builder.setFamily(ByteString.copyFrom(family));
+    builder.setDstSft(dstSFT);
+    builder.setNonceGroup(nonceGroup);
+    builder.setNonce(nonce);
+    return builder.build();
+  }
+
   /**
    * Create a protocol buffer MoveRegionRequest
    * @param encodedRegionName
@@ -1220,6 +1236,17 @@ public final class RequestConverter {
     return builder.build();
   }
 
+  public static ModifyTableStoreFileTrackerRequest buildModifyTableStoreFileTrackerRequest(
+    final TableName tableName, final String dstSFT, final long nonceGroup, final long nonce) {
+    ModifyTableStoreFileTrackerRequest.Builder builder =
+      ModifyTableStoreFileTrackerRequest.newBuilder();
+    builder.setTableName(ProtobufUtil.toProtoTableName((tableName)));
+    builder.setDstSft(dstSFT);
+    builder.setNonceGroup(nonceGroup);
+    builder.setNonce(nonce);
+    return builder.build();
+  }
+
   /**
    * Creates a protocol buffer GetTableDescriptorsRequest
    *
diff --git a/hbase-protocol-shaded/src/main/protobuf/server/master/Master.proto b/hbase-protocol-shaded/src/main/protobuf/server/master/Master.proto
index c502f45..dce0e68 100644
--- a/hbase-protocol-shaded/src/main/protobuf/server/master/Master.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/server/master/Master.proto
@@ -732,6 +732,29 @@ message BalancerRejectionsResponse {
   repeated BalancerRejection balancer_rejection = 1;
 }
 
+message ModifyTableStoreFileTrackerRequest {
+  required TableName table_Name = 1;
+  required string dst_sft = 2;
+  optional uint64 nonce_group = 3 [default = 0];
+  optional uint64 nonce = 4 [default = 0];
+}
+
+message ModifyTableStoreFileTrackerResponse {
+  optional uint64 proc_id = 1;
+}
+
+message ModifyColumnStoreFileTrackerRequest {
+  required TableName table_Name = 1;
+  required bytes family = 2;
+  required string dst_sft = 3;
+  optional uint64 nonce_group = 4 [default = 0];
+  optional uint64 nonce = 5 [default = 0];
+}
+
+message ModifyColumnStoreFileTrackerResponse {
+  optional uint64 proc_id = 1;
+}
+
 service MasterService {
   /** Used by the client to get the number of regions that have received the updated schema */
   rpc GetSchemaAlterStatus(GetSchemaAlterStatusRequest)
@@ -1165,6 +1188,12 @@ service MasterService {
 
   rpc GetLogEntries(LogRequest)
     returns(LogEntry);
+
+  rpc ModifyTableStoreFileTracker(ModifyTableStoreFileTrackerRequest)
+    returns(ModifyTableStoreFileTrackerResponse);
+
+  rpc ModifyColumnStoreFileTracker(ModifyColumnStoreFileTrackerRequest)
+    returns(ModifyColumnStoreFileTrackerResponse);
 }
 
 // HBCK Service definitions.
diff --git a/hbase-protocol-shaded/src/main/protobuf/server/master/MasterProcedure.proto b/hbase-protocol-shaded/src/main/protobuf/server/master/MasterProcedure.proto
index 4f92e950..7795e2e 100644
--- a/hbase-protocol-shaded/src/main/protobuf/server/master/MasterProcedure.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/server/master/MasterProcedure.proto
@@ -658,3 +658,18 @@ message ModifyTableDescriptorStateData {
   required TableName table_name = 1;
   optional TableSchema modified_table_schema = 2;
 }
+
+enum ModifyStoreFileTrackerState {
+  MODIFY_STORE_FILE_TRACKER_FINISH_PREVIOUS_MIGRATION = 1;
+  MODIFY_STORE_FILE_TRACKER_START_MIGRATION = 2;
+  MODIFY_STORE_FILE_TRACKER_FINISH_MIGRATION = 3;
+}
+
+message ModifyStoreFileTrackerStateData {
+  required TableName table_name = 1;
+  required string dst_sft = 2;
+}
+
+message ModifyColumnFamilyStoreFileTrackerStateData {
+  required bytes family = 1;
+}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
index 076c18a..877b722 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
@@ -252,6 +252,58 @@ public interface MasterObserver {
     throws IOException {}
 
   /**
+   * Called prior to modifying a table's store file tracker. Called as part of modify
+   * table store file tracker RPC call.
+   * @param ctx the environment to interact with the framework and master
+   * @param tableName the name of the table
+   * @param dstSFT the store file tracker
+   * @return the store file tracker
+   */
+  default String preModifyTableStoreFileTracker(
+    final ObserverContext<MasterCoprocessorEnvironment> ctx, final TableName tableName,
+    String dstSFT) throws IOException {
+    return dstSFT;
+  }
+
+  /**
+   * Called after modifying a table's store file tracker. Called as part of modify
+   * table store file tracker RPC call.
+   * @param ctx the environment to interact with the framework and master
+   * @param tableName the name of the table
+   * @param dstSFT the store file tracker
+   */
+  default void postModifyTableStoreFileTracker(
+    final ObserverContext<MasterCoprocessorEnvironment> ctx, final TableName tableName,
+    String dstSFT) throws IOException {}
+
+  /**
+   * Called prior to modifying a family's store file tracker. Called as part of modify family store
+   * file tracker RPC call.
+   * @param ctx the environment to interact with the framework and master
+   * @param tableName the name of the table
+   * @param family the column family
+   * @param dstSFT the store file tracker
+   * @return the store file tracker
+   */
+  default String preModifyColumnFamilyStoreFileTracker(
+    final ObserverContext<MasterCoprocessorEnvironment> ctx, final TableName tableName,
+    final byte[] family, String dstSFT) throws IOException {
+    return dstSFT;
+  }
+
+  /**
+   * Called after modifying a family store file tracker. Called as part of modify family store
+   * file tracker RPC call.
+   * @param ctx the environment to interact with the framework and master
+   * @param tableName the name of the table
+   * @param family the column family
+   * @param dstSFT the store file tracker
+   */
+  default void postModifyColumnFamilyStoreFileTracker(
+    final ObserverContext<MasterCoprocessorEnvironment> ctx, final TableName tableName,
+    final byte[] family, String dstSFT) throws IOException {}
+
+  /**
    * Called prior to modifying a table's properties.  Called as part of modify
    * table procedure and it is async to the modify table RPC call.
    *
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 49c2f93..2266eda 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
@@ -196,6 +196,8 @@ import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshotNotifierFactory;
 import org.apache.hadoop.hbase.quotas.SpaceViolationPolicy;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.NoSuchColumnFamilyException;
+import org.apache.hadoop.hbase.regionserver.storefiletracker.ModifyColumnFamilyStoreFileTrackerProcedure;
+import org.apache.hadoop.hbase.regionserver.storefiletracker.ModifyTableStoreFileTrackerProcedure;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationLoadSource;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
@@ -2454,6 +2456,33 @@ public class HMaster extends HBaseServerBase<MasterRpcServices> implements Maste
     }, nonceGroup, nonce, true);
   }
 
+
+  @Override
+  public long modifyColumnStoreFileTracker(TableName tableName, byte[] family, String dstSFT,
+    long nonceGroup, long nonce) throws IOException {
+    checkInitialized();
+    return MasterProcedureUtil
+      .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
+
+        @Override
+        protected void run() throws IOException {
+          String sft = getMaster().getMasterCoprocessorHost()
+            .preModifyColumnFamilyStoreFileTracker(tableName, family, dstSFT);
+          LOG.info("{} modify column {} store file tracker of table {} to {}",
+            getClientIdAuditPrefix(), Bytes.toStringBinary(family), tableName, sft);
+          submitProcedure(new ModifyColumnFamilyStoreFileTrackerProcedure(
+            procedureExecutor.getEnvironment(), tableName, family, sft));
+          getMaster().getMasterCoprocessorHost().postModifyColumnFamilyStoreFileTracker(tableName,
+            family, dstSFT);
+        }
+
+        @Override
+        protected String getDescription() {
+          return "ModifyColumnFamilyStoreFileTrackerProcedure";
+        }
+      });
+  }
+
   @Override
   public long deleteColumn(final TableName tableName, final byte[] columnName,
       final long nonceGroup, final long nonce) throws IOException {
@@ -2619,6 +2648,31 @@ public class HMaster extends HBaseServerBase<MasterRpcServices> implements Maste
 
   }
 
+  @Override
+  public long modifyTableStoreFileTracker(TableName tableName, String dstSFT, long nonceGroup,
+    long nonce) throws IOException {
+    checkInitialized();
+    return MasterProcedureUtil
+      .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
+
+        @Override
+        protected void run() throws IOException {
+          String sft = getMaster().getMasterCoprocessorHost()
+            .preModifyTableStoreFileTracker(tableName, dstSFT);
+          LOG.info("{} modify table store file tracker of table {} to {}", getClientIdAuditPrefix(),
+            tableName, sft);
+          submitProcedure(new ModifyTableStoreFileTrackerProcedure(
+            procedureExecutor.getEnvironment(), tableName, sft));
+          getMaster().getMasterCoprocessorHost().postModifyTableStoreFileTracker(tableName, sft);
+        }
+
+        @Override
+        protected String getDescription() {
+          return "ModifyTableStoreFileTrackerProcedure";
+        }
+      });
+  }
+
   public long restoreSnapshot(final SnapshotDescription snapshotDesc, final long nonceGroup,
     final long nonce, final boolean restoreAcl, final String customSFT) throws IOException {
     checkInitialized();
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
index 3c1fcec..8037fa1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
@@ -488,6 +488,55 @@ public class MasterCoprocessorHost
     });
   }
 
+  public String preModifyTableStoreFileTracker(final TableName tableName, final String dstSFT)
+    throws IOException {
+    if (coprocEnvironments.isEmpty()) {
+      return dstSFT;
+    }
+    return execOperationWithResult(
+      new ObserverOperationWithResult<MasterObserver, String>(masterObserverGetter, dstSFT) {
+        @Override
+        protected String call(MasterObserver observer) throws IOException {
+          return observer.preModifyTableStoreFileTracker(this, tableName, getResult());
+        }
+      });
+  }
+
+  public void postModifyTableStoreFileTracker(final TableName tableName, final String dstSFT)
+    throws IOException {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
+      @Override
+      public void call(MasterObserver observer) throws IOException {
+        observer.postModifyTableStoreFileTracker(this, tableName, dstSFT);
+      }
+    });
+  }
+
+  public String preModifyColumnFamilyStoreFileTracker(final TableName tableName,
+    final byte[] family, final String dstSFT) throws IOException {
+    if (coprocEnvironments.isEmpty()) {
+      return dstSFT;
+    }
+    return execOperationWithResult(
+      new ObserverOperationWithResult<MasterObserver, String>(masterObserverGetter, dstSFT) {
+        @Override
+        protected String call(MasterObserver observer) throws IOException {
+          return observer.preModifyColumnFamilyStoreFileTracker(this, tableName, family,
+            getResult());
+        }
+      });
+  }
+
+  public void postModifyColumnFamilyStoreFileTracker(final TableName tableName, final byte[] family,
+    final String dstSFT) throws IOException {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
+      @Override
+      public void call(MasterObserver observer) throws IOException {
+        observer.postModifyColumnFamilyStoreFileTracker(this, tableName, family, dstSFT);
+      }
+    });
+  }
+
   public void preModifyTableAction(final TableName tableName,
     final TableDescriptor currentDescriptor, final TableDescriptor newDescriptor, final User user)
     throws IOException {
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 78fb390..d53a271 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
@@ -287,10 +287,14 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTable
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnStoreFileTrackerRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnStoreFileTrackerResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableStoreFileTrackerRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableStoreFileTrackerResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest;
@@ -1478,6 +1482,20 @@ public class MasterRpcServices extends HBaseRpcServicesBase<HMaster>
   }
 
   @Override
+  public ModifyColumnStoreFileTrackerResponse modifyColumnStoreFileTracker(
+    RpcController controller, ModifyColumnStoreFileTrackerRequest req)
+    throws ServiceException {
+    try {
+      long procId =
+        server.modifyColumnStoreFileTracker(ProtobufUtil.toTableName(req.getTableName()),
+          req.getFamily().toByteArray(), req.getDstSft(), req.getNonceGroup(), req.getNonce());
+      return ModifyColumnStoreFileTrackerResponse.newBuilder().setProcId(procId).build();
+    } catch (IOException ioe) {
+      throw new ServiceException(ioe);
+    }
+  }
+
+  @Override
   public ModifyNamespaceResponse modifyNamespace(RpcController controller,
       ModifyNamespaceRequest request) throws ServiceException {
     try {
@@ -1507,6 +1525,18 @@ public class MasterRpcServices extends HBaseRpcServicesBase<HMaster>
   }
 
   @Override
+  public ModifyTableStoreFileTrackerResponse modifyTableStoreFileTracker(RpcController controller,
+    ModifyTableStoreFileTrackerRequest req) throws ServiceException {
+    try {
+      long procId = server.modifyTableStoreFileTracker(ProtobufUtil.toTableName(req.getTableName()),
+        req.getDstSft(), req.getNonceGroup(), req.getNonce());
+      return ModifyTableStoreFileTrackerResponse.newBuilder().setProcId(procId).build();
+    } catch (IOException ioe) {
+      throw new ServiceException(ioe);
+    }
+  }
+
+  @Override
   public MoveRegionResponse moveRegion(RpcController controller,
       MoveRegionRequest req) throws ServiceException {
     final byte [] encodedRegionName = req.getRegion().getValue().toByteArray();
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
index f24ecd4..85d9a11 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
@@ -221,6 +221,15 @@ public interface MasterServices extends Server {
       throws IOException;
 
   /**
+   * Modify the store file tracker of an existing table
+   */
+  long modifyTableStoreFileTracker(
+    final TableName tableName,
+    final String dstSFT,
+    final long nonceGroup,
+    final long nonce) throws IOException;
+
+  /**
    * Enable an existing table
    * @param tableName The table name
    * @param nonceGroup
@@ -276,6 +285,16 @@ public interface MasterServices extends Server {
       throws IOException;
 
   /**
+   * Modify the store file tracker of an existing column in an existing table
+   */
+  long modifyColumnStoreFileTracker(
+    final TableName tableName,
+    final byte[] family,
+    final String dstSFT,
+    final long nonceGroup,
+    final long nonce) throws IOException;
+
+  /**
    * Delete a column from an existing table
    * @param tableName The table name
    * @param columnName The column name
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 bf055e5..3467fd5 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
@@ -37,6 +37,7 @@ 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.regionserver.storefiletracker.StoreFileTrackerValidationUtils;
 import org.apache.hadoop.hbase.rsgroup.RSGroupInfo;
 import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.FSTableDescriptors;
@@ -273,7 +274,8 @@ public class CreateTableProcedure
     }
 
     // check for store file tracker configurations
-    StoreFileTrackerFactory.checkForCreateTable(env.getMasterConfiguration(), tableDescriptor);
+    StoreFileTrackerValidationUtils.checkForCreateTable(env.getMasterConfiguration(),
+      tableDescriptor);
 
     return true;
   }
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 4291a8f..9237f39 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
@@ -38,7 +38,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.regionserver.storefiletracker.StoreFileTrackerValidationUtils;
 import org.apache.hadoop.hbase.rsgroup.RSGroupInfo;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -326,7 +326,7 @@ public class ModifyTableProcedure
     }
 
     // check for store file tracker configurations
-    StoreFileTrackerFactory.checkForModifyTable(env.getMasterConfiguration(),
+    StoreFileTrackerValidationUtils.checkForModifyTable(env.getMasterConfiguration(),
       unmodifiedTableDescriptor, 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 76cd1bf..14626eb 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,7 +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.regionserver.storefiletracker.StoreFileTrackerValidationUtils;
 import org.apache.hadoop.hbase.security.AccessDeniedException;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.access.AccessChecker;
@@ -884,8 +884,8 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
     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());
+    StoreFileTrackerValidationUtils.validatePreRestoreSnapshot(
+      master.getTableDescriptors().get(tableName), snapshotTableDesc, master.getConfiguration());
 
     if (master.getTableStateManager().isTableState(
       TableName.valueOf(snapshot.getTable()), TableState.State.ENABLED)) {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/ModifyColumnFamilyStoreFileTrackerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/ModifyColumnFamilyStoreFileTrackerProcedure.java
new file mode 100644
index 0000000..1ecfee2
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/ModifyColumnFamilyStoreFileTrackerProcedure.java
@@ -0,0 +1,107 @@
+/**
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseIOException;
+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.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
+import org.apache.hadoop.hbase.regionserver.NoSuchColumnFamilyException;
+import org.apache.hadoop.hbase.regionserver.StoreUtils;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyColumnFamilyStoreFileTrackerStateData;
+
+@InterfaceAudience.Private
+public class ModifyColumnFamilyStoreFileTrackerProcedure extends ModifyStoreFileTrackerProcedure {
+
+  private byte[] family;
+
+  public ModifyColumnFamilyStoreFileTrackerProcedure() {
+  }
+
+  public ModifyColumnFamilyStoreFileTrackerProcedure(MasterProcedureEnv env, TableName tableName,
+    byte[] family, String dstSFT) throws HBaseIOException {
+    super(env, tableName, dstSFT);
+    this.family = family;
+  }
+
+  @Override
+  protected void preCheck(TableDescriptor current) throws IOException {
+    if (!current.hasColumnFamily(family)) {
+      throw new NoSuchColumnFamilyException(
+        Bytes.toStringBinary(family) + " does not exist for table " + current.getTableName());
+    }
+  }
+
+  @Override
+  protected Configuration createConf(Configuration conf, TableDescriptor current) {
+    ColumnFamilyDescriptor cfd = current.getColumnFamily(family);
+    return StoreUtils.createStoreConfiguration(conf, current, cfd);
+  }
+
+  @Override
+  protected TableDescriptor createRestoreTableDescriptor(TableDescriptor current,
+    String restoreSFT) {
+    ColumnFamilyDescriptor cfd =
+      ColumnFamilyDescriptorBuilder.newBuilder(current.getColumnFamily(family))
+        .setConfiguration(StoreFileTrackerFactory.TRACKER_IMPL, restoreSFT).build();
+    return TableDescriptorBuilder.newBuilder(current).modifyColumnFamily(cfd).build();
+  }
+
+  @Override
+  protected TableDescriptor createMigrationTableDescriptor(Configuration conf,
+    TableDescriptor current) {
+    ColumnFamilyDescriptorBuilder builder =
+      ColumnFamilyDescriptorBuilder.newBuilder(current.getColumnFamily(family));
+    migrate(conf, builder::setConfiguration);
+    return TableDescriptorBuilder.newBuilder(current).modifyColumnFamily(builder.build()).build();
+  }
+
+  @Override
+  protected TableDescriptor createFinishTableDescriptor(TableDescriptor current) {
+    ColumnFamilyDescriptorBuilder builder =
+      ColumnFamilyDescriptorBuilder.newBuilder(current.getColumnFamily(family));
+    finish(builder::setConfiguration, builder::removeConfiguration);
+    return TableDescriptorBuilder.newBuilder(current).modifyColumnFamily(builder.build()).build();
+  }
+
+  @Override
+  protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException {
+    super.serializeStateData(serializer);
+    serializer.serialize(ModifyColumnFamilyStoreFileTrackerStateData.newBuilder()
+      .setFamily(ByteString.copyFrom(family)).build());
+  }
+
+  @Override
+  protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException {
+    super.deserializeStateData(serializer);
+    ModifyColumnFamilyStoreFileTrackerStateData data =
+      serializer.deserialize(ModifyColumnFamilyStoreFileTrackerStateData.class);
+    this.family = data.getFamily().toByteArray();
+  }
+}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/ModifyStoreFileTrackerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/ModifyStoreFileTrackerProcedure.java
new file mode 100644
index 0000000..a7d8e70
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/ModifyStoreFileTrackerProcedure.java
@@ -0,0 +1,271 @@
+/**
+ * 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.function.BiConsumer;
+import java.util.function.Consumer;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.HBaseIOException;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.TableNotEnabledException;
+import org.apache.hadoop.hbase.TableNotFoundException;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.master.procedure.AbstractStateMachineTableProcedure;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.master.procedure.ModifyTableProcedure;
+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.ModifyStoreFileTrackerState;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyStoreFileTrackerStateData;
+
+/**
+ * This procedure is used to change the store file tracker implementation.
+ * <p/>
+ * Typically we need to schedule two {@link ModifyTableProcedure} (or three if the table is already
+ * in {@code MIGRATION} but the {@code dstSFT} is not what we expected) to do this, so we introduce
+ * this procedure to simplify the work of our users.
+ */
+@InterfaceAudience.Private
+public abstract class ModifyStoreFileTrackerProcedure
+  extends AbstractStateMachineTableProcedure<ModifyStoreFileTrackerState> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ModifyStoreFileTrackerProcedure.class);
+
+  private TableName tableName;
+
+  private String dstSFT;
+
+  protected ModifyStoreFileTrackerProcedure() {
+  }
+
+  protected ModifyStoreFileTrackerProcedure(MasterProcedureEnv env, TableName tableName,
+    String dstSFT) throws HBaseIOException {
+    super(env);
+    checkDstSFT(dstSFT);
+    this.tableName = tableName;
+    this.dstSFT = dstSFT;
+    preflightChecks(env, true);
+  }
+
+  private void checkDstSFT(String dstSFT) throws DoNotRetryIOException {
+    if (MigrationStoreFileTracker.class
+      .isAssignableFrom(StoreFileTrackerFactory.getTrackerClass(dstSFT))) {
+      throw new DoNotRetryIOException("Do not need to transfer to " + dstSFT);
+    }
+  }
+
+  @Override
+  public TableName getTableName() {
+    return tableName;
+  }
+
+  @Override
+  public TableOperationType getTableOperationType() {
+    return TableOperationType.EDIT;
+  }
+
+  private enum StoreFileTrackerState {
+    NEED_FINISH_PREVIOUS_MIGRATION_FIRST, NEED_START_MIGRATION, NEED_FINISH_MIGRATION,
+    ALREADY_FINISHED
+  }
+
+  private StoreFileTrackerState checkState(Configuration conf, String dstSFT) {
+    // there are 4 possible conditions:
+    // 1. The table or family has already made use of the dstSFT. In this way we just finish the
+    // procedure.
+    // 2. The table or family is not using the dstSFT but also not using migration SFT,
+    // then we just go to the MODIFY_STORE_FILE_TRACKER_MIGRATION state.
+    // 3. The table or family has already been using migration SFT and the dst SFT is what we
+    // expect, just go to MODIFY_STORE_FILE_TRACKER_FINISH.
+    // 4. The table or family is using migration SFT and the dst SFT is not what we
+    // expect, then need to schedule a MTP to change it to the dst SFT of the current migration
+    // SFT first, and then go to MODIFY_STORE_FILE_TRACKER_MIGRATION.
+    Class<? extends StoreFileTracker> clazz = StoreFileTrackerFactory.getTrackerClass(conf);
+    Class<? extends StoreFileTracker> dstSFTClass = StoreFileTrackerFactory.getTrackerClass(dstSFT);
+    if (clazz.equals(dstSFTClass)) {
+      return StoreFileTrackerState.ALREADY_FINISHED;
+    }
+    if (!MigrationStoreFileTracker.class.isAssignableFrom(clazz)) {
+      return StoreFileTrackerState.NEED_START_MIGRATION;
+    }
+    Class<? extends StoreFileTracker> currentDstSFT = StoreFileTrackerFactory
+      .getStoreFileTrackerClassForMigration(conf, MigrationStoreFileTracker.DST_IMPL);
+    if (currentDstSFT.equals(dstSFTClass)) {
+      return StoreFileTrackerState.NEED_FINISH_MIGRATION;
+    } else {
+      return StoreFileTrackerState.NEED_FINISH_PREVIOUS_MIGRATION_FIRST;
+    }
+  }
+
+  private final String getRestoreSFT(Configuration conf) {
+    Class<? extends StoreFileTracker> currentDstSFT = StoreFileTrackerFactory
+      .getStoreFileTrackerClassForMigration(conf, MigrationStoreFileTracker.DST_IMPL);
+    return StoreFileTrackerFactory.getStoreFileTrackerName(currentDstSFT);
+  }
+
+  protected abstract void preCheck(TableDescriptor current) throws IOException;
+
+  protected abstract Configuration createConf(Configuration conf, TableDescriptor current);
+
+  protected abstract TableDescriptor createRestoreTableDescriptor(TableDescriptor current,
+    String restoreSFT);
+
+  private Flow preCheckAndTryRestoreSFT(MasterProcedureEnv env) throws IOException {
+    // Checks whether the table exists
+    if (!env.getMasterServices().getTableDescriptors().exists(getTableName())) {
+      throw new TableNotFoundException(getTableName());
+    }
+    if (!isTableEnabled(env)) {
+      throw new TableNotEnabledException(tableName);
+    }
+    TableDescriptor current = env.getMasterServices().getTableDescriptors().get(tableName);
+    preCheck(current);
+    Configuration conf = createConf(env.getMasterConfiguration(), current);
+    StoreFileTrackerState state = checkState(conf, dstSFT);
+    switch (state) {
+      case NEED_FINISH_PREVIOUS_MIGRATION_FIRST:
+        TableDescriptor td = createRestoreTableDescriptor(current, getRestoreSFT(conf));
+        addChildProcedure(new ModifyTableProcedure(env, td));
+        setNextState(
+          ModifyStoreFileTrackerState.MODIFY_STORE_FILE_TRACKER_FINISH_PREVIOUS_MIGRATION);
+        return Flow.HAS_MORE_STATE;
+      case NEED_START_MIGRATION:
+        setNextState(ModifyStoreFileTrackerState.MODIFY_STORE_FILE_TRACKER_START_MIGRATION);
+        return Flow.HAS_MORE_STATE;
+      case NEED_FINISH_MIGRATION:
+        setNextState(ModifyStoreFileTrackerState.MODIFY_STORE_FILE_TRACKER_FINISH_MIGRATION);
+        return Flow.HAS_MORE_STATE;
+      case ALREADY_FINISHED:
+        return Flow.NO_MORE_STATE;
+      default:
+        throw new UnsupportedOperationException("unhandled state=" + state);
+    }
+  }
+
+  protected abstract TableDescriptor createMigrationTableDescriptor(Configuration conf,
+    TableDescriptor current);
+
+  protected final void migrate(Configuration conf, BiConsumer<String, String> setValue) {
+    setValue.accept(StoreFileTrackerFactory.TRACKER_IMPL,
+      StoreFileTrackerFactory.Trackers.MIGRATION.name());
+    setValue.accept(MigrationStoreFileTracker.SRC_IMPL,
+      StoreFileTrackerFactory.getStoreFileTrackerName(conf));
+    setValue.accept(MigrationStoreFileTracker.DST_IMPL, dstSFT);
+  }
+
+  protected abstract TableDescriptor createFinishTableDescriptor(TableDescriptor current);
+
+  protected final void finish(BiConsumer<String, String> setValue, Consumer<String> removeValue) {
+    setValue.accept(StoreFileTrackerFactory.TRACKER_IMPL, dstSFT);
+    removeValue.accept(MigrationStoreFileTracker.SRC_IMPL);
+    removeValue.accept(MigrationStoreFileTracker.DST_IMPL);
+  }
+
+  private void migrate(MasterProcedureEnv env) throws IOException {
+    TableDescriptor current = env.getMasterServices().getTableDescriptors().get(tableName);
+    TableDescriptor td = createMigrationTableDescriptor(env.getMasterConfiguration(), current);
+    addChildProcedure(new ModifyTableProcedure(env, td));
+    setNextState(ModifyStoreFileTrackerState.MODIFY_STORE_FILE_TRACKER_FINISH_MIGRATION);
+  }
+
+  private void finish(MasterProcedureEnv env) throws IOException {
+    TableDescriptor current = env.getMasterServices().getTableDescriptors().get(tableName);
+    TableDescriptor td = createFinishTableDescriptor(current);
+    addChildProcedure(new ModifyTableProcedure(env, td));
+  }
+
+  @Override
+  protected Flow executeFromState(MasterProcedureEnv env, ModifyStoreFileTrackerState state)
+    throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException {
+    try {
+      switch (state) {
+        case MODIFY_STORE_FILE_TRACKER_FINISH_PREVIOUS_MIGRATION:
+          return preCheckAndTryRestoreSFT(env);
+        case MODIFY_STORE_FILE_TRACKER_START_MIGRATION:
+          migrate(env);
+          return Flow.HAS_MORE_STATE;
+        case MODIFY_STORE_FILE_TRACKER_FINISH_MIGRATION:
+          finish(env);
+          return Flow.NO_MORE_STATE;
+        default:
+          throw new UnsupportedOperationException("unhandled state=" + state);
+      }
+    } catch (IOException e) {
+      if (isRollbackSupported(state)) {
+        setFailure("master-modify-SFT", e);
+      } else {
+        LOG.warn("Retriable error trying to modify SFT for table={} (in state={})", getTableName(),
+          state, e);
+      }
+    }
+    return Flow.HAS_MORE_STATE;
+  }
+
+  @Override
+  protected void rollbackState(MasterProcedureEnv env, ModifyStoreFileTrackerState state)
+    throws IOException, InterruptedException {
+    if (isRollbackSupported(state)) {
+      return;
+    }
+    throw new UnsupportedOperationException("unhandled state=" + state);
+  }
+
+  @Override
+  protected ModifyStoreFileTrackerState getState(int stateId) {
+    return ModifyStoreFileTrackerState.forNumber(stateId);
+  }
+
+  @Override
+  protected int getStateId(ModifyStoreFileTrackerState state) {
+    return state.getNumber();
+  }
+
+  @Override
+  protected ModifyStoreFileTrackerState getInitialState() {
+    return ModifyStoreFileTrackerState.MODIFY_STORE_FILE_TRACKER_FINISH_PREVIOUS_MIGRATION;
+  }
+
+  @Override
+  protected boolean isRollbackSupported(ModifyStoreFileTrackerState state) {
+    return state == ModifyStoreFileTrackerState.MODIFY_STORE_FILE_TRACKER_FINISH_PREVIOUS_MIGRATION;
+  }
+
+  @Override
+  protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException {
+    super.serializeStateData(serializer);
+    serializer.serialize(ModifyStoreFileTrackerStateData.newBuilder()
+      .setTableName(ProtobufUtil.toProtoTableName(tableName)).setDstSft(dstSFT).build());
+  }
+
+  @Override
+  protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException {
+    super.deserializeStateData(serializer);
+    ModifyStoreFileTrackerStateData data =
+      serializer.deserialize(ModifyStoreFileTrackerStateData.class);
+    this.tableName = ProtobufUtil.toTableName(data.getTableName());
+    this.dstSFT = data.getDstSft();
+  }
+}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/ModifyTableStoreFileTrackerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/ModifyTableStoreFileTrackerProcedure.java
new file mode 100644
index 0000000..096f38f
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/ModifyTableStoreFileTrackerProcedure.java
@@ -0,0 +1,71 @@
+/**
+ * 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.CompoundConfiguration;
+import org.apache.hadoop.hbase.HBaseIOException;
+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.master.procedure.MasterProcedureEnv;
+import org.apache.yetus.audience.InterfaceAudience;
+
+@InterfaceAudience.Private
+public class ModifyTableStoreFileTrackerProcedure extends ModifyStoreFileTrackerProcedure {
+
+  public ModifyTableStoreFileTrackerProcedure() {
+  }
+
+  public ModifyTableStoreFileTrackerProcedure(MasterProcedureEnv env, TableName tableName,
+    String dstSFT) throws HBaseIOException {
+    super(env, tableName, dstSFT);
+  }
+
+  @Override
+  protected void preCheck(TableDescriptor current) {
+  }
+
+  @Override
+  protected Configuration createConf(Configuration conf, TableDescriptor current) {
+    return new CompoundConfiguration().add(conf).addBytesMap(current.getValues());
+  }
+
+  @Override
+  protected TableDescriptor createRestoreTableDescriptor(TableDescriptor current,
+    String restoreSFT) {
+    return TableDescriptorBuilder.newBuilder(current)
+      .setValue(StoreFileTrackerFactory.TRACKER_IMPL, restoreSFT).build();
+  }
+
+  @Override
+  protected TableDescriptor createMigrationTableDescriptor(Configuration conf,
+    TableDescriptor current) {
+    TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(current);
+    migrate(conf, builder::setValue);
+    return builder.build();
+  }
+
+  @Override
+  protected TableDescriptor createFinishTableDescriptor(TableDescriptor current) {
+    TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(current);
+    finish(builder::setValue, builder::removeValue);
+    return builder.build();
+  }
+
+}
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 61a71c2..7f545e4 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,21 +15,17 @@
  */
 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.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.snapshot.RestoreSnapshotException;
 import org.apache.hadoop.hbase.util.ReflectionUtils;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
@@ -103,6 +99,20 @@ public final class StoreFileTrackerFactory {
     }
   }
 
+  public static Class<? extends StoreFileTracker> getTrackerClass(String trackerNameOrClass) {
+    try {
+      Trackers tracker = Trackers.valueOf(trackerNameOrClass.toUpperCase());
+      return tracker.clazz;
+    } catch (IllegalArgumentException e) {
+      // Fall back to them specifying a class name
+      try {
+        return Class.forName(trackerNameOrClass).asSubclass(StoreFileTracker.class);
+      } catch (ClassNotFoundException e1) {
+        throw new RuntimeException(e1);
+      }
+    }
+  }
+
   public static StoreFileTracker create(Configuration conf, boolean isPrimaryReplica,
     StoreContext ctx) {
     Class<? extends StoreFileTracker> tracker = getTrackerClass(conf);
@@ -174,178 +184,4 @@ public final class StoreFileTrackerFactory {
     }
     return descriptor;
   }
-
-  // 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());
-          }
-        }
-      }
-    }
-  }
-
-  /**
-   * 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/regionserver/storefiletracker/StoreFileTrackerValidationUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerValidationUtils.java
new file mode 100644
index 0000000..e6f6e85
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerValidationUtils.java
@@ -0,0 +1,212 @@
+/**
+ * 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 org.apache.hadoop.conf.Configuration;
+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.regionserver.StoreUtils;
+import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.Trackers;
+import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
+import org.apache.yetus.audience.InterfaceAudience;
+
+@InterfaceAudience.Private
+public final class StoreFileTrackerValidationUtils {
+
+  private StoreFileTrackerValidationUtils() {
+  }
+
+  // should not use MigrationStoreFileTracker for new family
+  private static void checkForNewFamily(Configuration conf, TableDescriptor table,
+    ColumnFamilyDescriptor family) throws IOException {
+    Configuration mergedConf = StoreUtils.createStoreConfiguration(conf, table, family);
+    Class<? extends StoreFileTracker> tracker = StoreFileTrackerFactory.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 = StoreUtils.createStoreConfiguration(conf, oldTable, oldFamily);
+      Configuration newConf = StoreUtils.createStoreConfiguration(conf, newTable, newFamily);
+
+      Class<? extends StoreFileTracker> oldTracker =
+        StoreFileTrackerFactory.getTrackerClass(oldConf);
+      Class<? extends StoreFileTracker> newTracker =
+        StoreFileTrackerFactory.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 " +
+              StoreFileTrackerFactory.getStoreFileTrackerName(oldSrcTracker) + " to " +
+              StoreFileTrackerFactory.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 " +
+              StoreFileTrackerFactory.getStoreFileTrackerName(oldDstTracker) + " to " +
+              StoreFileTrackerFactory.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 " +
+              StoreFileTrackerFactory.getStoreFileTrackerName(oldDstTracker) + " but got " +
+              StoreFileTrackerFactory.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 " +
+                StoreFileTrackerFactory.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 " +
+              StoreFileTrackerFactory.getStoreFileTrackerName(oldTracker) + " first but got " +
+              StoreFileTrackerFactory.getStoreFileTrackerName(newSrcTracker) +
+              " when migrating from " +
+              StoreFileTrackerFactory.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 " +
+              StoreFileTrackerFactory.getStoreFileTrackerName(newSrcTracker) + " for family " +
+              newFamily.getNameAsString() + " of table " + newTable.getTableName());
+          }
+        }
+      }
+    }
+  }
+
+  /**
+   * 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/security/access/AccessController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
index 0b8d7e4..1594e13 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
@@ -899,6 +899,23 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor,
   }
 
   @Override
+  public String preModifyTableStoreFileTracker(ObserverContext<MasterCoprocessorEnvironment> c,
+    TableName tableName, String dstSFT) throws IOException {
+    requirePermission(c, "modifyTableStoreFileTracker", tableName, null, null, Action.ADMIN,
+      Action.CREATE);
+    return dstSFT;
+  }
+
+  @Override
+  public String preModifyColumnFamilyStoreFileTracker(
+    ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName, byte[] family,
+    String dstSFT) throws IOException {
+    requirePermission(c, "modifyColumnFamilyStoreFileTracker", tableName, family, null,
+      Action.ADMIN, Action.CREATE);
+    return dstSFT;
+  }
+
+  @Override
   public void postModifyTable(ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName,
     TableDescriptor oldDesc, TableDescriptor currentDesc) throws IOException {
     final Configuration conf = c.getEnvironment().getConfiguration();
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 c2de0fb..9657fda 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
@@ -18,14 +18,18 @@
 package org.apache.hadoop.hbase.client;
 
 import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.TRACKER_IMPL;
+import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThrows;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
 import java.util.List;
 import java.util.regex.Pattern;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionLocation;
@@ -35,6 +39,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.NoSuchColumnFamilyException;
 import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
@@ -393,4 +398,156 @@ public class TestAdmin3 extends TestAdminBase {
       ADMIN.deleteTable(tableName);
     }
   }
+
+  private static final String SRC_IMPL = "hbase.store.file-tracker.migration.src.impl";
+
+  private static final String DST_IMPL = "hbase.store.file-tracker.migration.dst.impl";
+
+  private void verifyModifyTableResult(TableName tableName, byte[] family, byte[] qual, byte[] row,
+    byte[] value, String sft) throws IOException {
+    TableDescriptor td = ADMIN.getDescriptor(tableName);
+    assertEquals(sft, td.getValue(StoreFileTrackerFactory.TRACKER_IMPL));
+    // no migration related configs
+    assertNull(td.getValue(SRC_IMPL));
+    assertNull(td.getValue(DST_IMPL));
+    try (Table table = TEST_UTIL.getConnection().getTable(tableName)) {
+      assertArrayEquals(value, table.get(new Get(row)).getValue(family, qual));
+    }
+  }
+
+  @Test
+  public void testModifyTableStoreFileTracker() throws IOException {
+    TableName tableName = TableName.valueOf(name.getMethodName());
+    byte[] family = Bytes.toBytes("info");
+    byte[] qual = Bytes.toBytes("q");
+    byte[] row = Bytes.toBytes(0);
+    byte[] value = Bytes.toBytes(1);
+    try (Table table = TEST_UTIL.createTable(tableName, family)) {
+      table.put(new Put(row).addColumn(family, qual, value));
+    }
+    // change to FILE
+    ADMIN.modifyTableStoreFileTracker(tableName, StoreFileTrackerFactory.Trackers.FILE.name());
+    verifyModifyTableResult(tableName, family, qual, row, value,
+      StoreFileTrackerFactory.Trackers.FILE.name());
+
+    // change to FILE again, should have no effect
+    ADMIN.modifyTableStoreFileTracker(tableName, StoreFileTrackerFactory.Trackers.FILE.name());
+    verifyModifyTableResult(tableName, family, qual, row, value,
+      StoreFileTrackerFactory.Trackers.FILE.name());
+
+    // change to MIGRATION, and then to FILE
+    ADMIN.modifyTable(TableDescriptorBuilder.newBuilder(ADMIN.getDescriptor(tableName))
+      .setValue(StoreFileTrackerFactory.TRACKER_IMPL,
+        StoreFileTrackerFactory.Trackers.MIGRATION.name())
+      .setValue(SRC_IMPL,
+        StoreFileTrackerFactory.Trackers.FILE.name())
+      .setValue(DST_IMPL,
+        StoreFileTrackerFactory.Trackers.DEFAULT.name())
+      .build());
+    ADMIN.modifyTableStoreFileTracker(tableName, StoreFileTrackerFactory.Trackers.FILE.name());
+    verifyModifyTableResult(tableName, family, qual, row, value,
+      StoreFileTrackerFactory.Trackers.FILE.name());
+
+    // change to MIGRATION, and then to DEFAULT
+    ADMIN.modifyTable(TableDescriptorBuilder.newBuilder(ADMIN.getDescriptor(tableName))
+      .setValue(StoreFileTrackerFactory.TRACKER_IMPL,
+        StoreFileTrackerFactory.Trackers.MIGRATION.name())
+      .setValue(SRC_IMPL,
+        StoreFileTrackerFactory.Trackers.FILE.name())
+      .setValue(DST_IMPL,
+        StoreFileTrackerFactory.Trackers.DEFAULT.name())
+      .build());
+    ADMIN.modifyTableStoreFileTracker(tableName, StoreFileTrackerFactory.Trackers.DEFAULT.name());
+    verifyModifyTableResult(tableName, family, qual, row, value,
+      StoreFileTrackerFactory.Trackers.DEFAULT.name());
+  }
+
+  private void verifyModifyColumnFamilyResult(TableName tableName, byte[] family, byte[] qual,
+    byte[] row, byte[] value, String sft) throws IOException {
+    TableDescriptor td = ADMIN.getDescriptor(tableName);
+    ColumnFamilyDescriptor cfd = td.getColumnFamily(family);
+    assertEquals(sft, cfd.getConfigurationValue(StoreFileTrackerFactory.TRACKER_IMPL));
+    // no migration related configs
+    assertNull(cfd.getConfigurationValue(SRC_IMPL));
+    assertNull(cfd.getConfigurationValue(DST_IMPL));
+    assertNull(cfd.getValue(SRC_IMPL));
+    assertNull(cfd.getValue(DST_IMPL));
+    try (Table table = TEST_UTIL.getConnection().getTable(tableName)) {
+      assertArrayEquals(value, table.get(new Get(row)).getValue(family, qual));
+    }
+  }
+
+  @Test
+  public void testModifyColumnFamilyStoreFileTracker() throws IOException {
+    TableName tableName = TableName.valueOf(name.getMethodName());
+    byte[] family = Bytes.toBytes("info");
+    byte[] qual = Bytes.toBytes("q");
+    byte[] row = Bytes.toBytes(0);
+    byte[] value = Bytes.toBytes(1);
+    try (Table table = TEST_UTIL.createTable(tableName, family)) {
+      table.put(new Put(row).addColumn(family, qual, value));
+    }
+    // change to FILE
+    ADMIN.modifyColumnFamilyStoreFileTracker(tableName, family,
+      StoreFileTrackerFactory.Trackers.FILE.name());
+    verifyModifyColumnFamilyResult(tableName, family, qual, row, value,
+      StoreFileTrackerFactory.Trackers.FILE.name());
+
+    // change to FILE again, should have no effect
+    ADMIN.modifyColumnFamilyStoreFileTracker(tableName, family,
+      StoreFileTrackerFactory.Trackers.FILE.name());
+    verifyModifyColumnFamilyResult(tableName, family, qual, row, value,
+      StoreFileTrackerFactory.Trackers.FILE.name());
+
+    // change to MIGRATION, and then to FILE
+    TableDescriptor current = ADMIN.getDescriptor(tableName);
+    ADMIN.modifyTable(TableDescriptorBuilder.newBuilder(current)
+      .modifyColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(current.getColumnFamily(family))
+        .setConfiguration(StoreFileTrackerFactory.TRACKER_IMPL,
+          StoreFileTrackerFactory.Trackers.MIGRATION.name())
+        .setConfiguration(SRC_IMPL, StoreFileTrackerFactory.Trackers.FILE.name())
+        .setConfiguration(DST_IMPL, StoreFileTrackerFactory.Trackers.DEFAULT.name()).build())
+      .build());
+    ADMIN.modifyColumnFamilyStoreFileTracker(tableName, family,
+      StoreFileTrackerFactory.Trackers.FILE.name());
+    verifyModifyColumnFamilyResult(tableName, family, qual, row, value,
+      StoreFileTrackerFactory.Trackers.FILE.name());
+
+    // change to MIGRATION, and then to DEFAULT
+    current = ADMIN.getDescriptor(tableName);
+    ADMIN.modifyTable(TableDescriptorBuilder.newBuilder(current)
+      .modifyColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(current.getColumnFamily(family))
+        .setConfiguration(StoreFileTrackerFactory.TRACKER_IMPL,
+          StoreFileTrackerFactory.Trackers.MIGRATION.name())
+        .setConfiguration(SRC_IMPL, StoreFileTrackerFactory.Trackers.FILE.name())
+        .setConfiguration(DST_IMPL, StoreFileTrackerFactory.Trackers.DEFAULT.name()).build())
+      .build());
+    ADMIN.modifyColumnFamilyStoreFileTracker(tableName, family,
+      StoreFileTrackerFactory.Trackers.DEFAULT.name());
+    verifyModifyColumnFamilyResult(tableName, family, qual, row, value,
+      StoreFileTrackerFactory.Trackers.DEFAULT.name());
+  }
+
+  @Test
+  public void testModifyStoreFileTrackerError() throws IOException {
+    TableName tableName = TableName.valueOf(name.getMethodName());
+    byte[] family = Bytes.toBytes("info");
+    TEST_UTIL.createTable(tableName, family).close();
+
+    // table not exists
+    assertThrows(TableNotFoundException.class,
+      () -> ADMIN.modifyTableStoreFileTracker(TableName.valueOf("whatever"),
+        StoreFileTrackerFactory.Trackers.FILE.name()));
+    // family not exists
+    assertThrows(NoSuchColumnFamilyException.class,
+      () -> ADMIN.modifyColumnFamilyStoreFileTracker(tableName, Bytes.toBytes("not_exists"),
+        StoreFileTrackerFactory.Trackers.FILE.name()));
+    // to migration
+    assertThrows(DoNotRetryIOException.class, () -> ADMIN.modifyTableStoreFileTracker(tableName,
+      StoreFileTrackerFactory.Trackers.MIGRATION.name()));
+    // disabled
+    ADMIN.disableTable(tableName);
+    assertThrows(TableNotEnabledException.class, () -> ADMIN.modifyTableStoreFileTracker(tableName,
+      StoreFileTrackerFactory.Trackers.FILE.name()));
+  }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminModifyStoreFileTracker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminModifyStoreFileTracker.java
new file mode 100644
index 0000000..c8821b6
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminModifyStoreFileTracker.java
@@ -0,0 +1,197 @@
+/**
+ * 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.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThrows;
+
+import java.io.IOException;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.TableNotEnabledException;
+import org.apache.hadoop.hbase.TableNotFoundException;
+import org.apache.hadoop.hbase.regionserver.NoSuchColumnFamilyException;
+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.FutureUtils;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+@Category({ LargeTests.class, ClientTests.class })
+public class TestAsyncAdminModifyStoreFileTracker extends TestAsyncAdminBase {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestAsyncAdminModifyStoreFileTracker.class);
+
+  private static final String SRC_IMPL = "hbase.store.file-tracker.migration.src.impl";
+
+  private static final String DST_IMPL = "hbase.store.file-tracker.migration.dst.impl";
+
+  private void verifyModifyTableResult(TableName tableName, byte[] family, byte[] qual, byte[] row,
+    byte[] value, String sft) throws IOException {
+    TableDescriptor td = admin.getDescriptor(tableName).join();
+    assertEquals(sft, td.getValue(StoreFileTrackerFactory.TRACKER_IMPL));
+    // no migration related configs
+    assertNull(td.getValue(SRC_IMPL));
+    assertNull(td.getValue(DST_IMPL));
+    try (Table table = TEST_UTIL.getConnection().getTable(tableName)) {
+      assertArrayEquals(value, table.get(new Get(row)).getValue(family, qual));
+    }
+  }
+
+  @Test
+  public void testModifyTableStoreFileTracker() throws IOException {
+    byte[] family = Bytes.toBytes("info");
+    byte[] qual = Bytes.toBytes("q");
+    byte[] row = Bytes.toBytes(0);
+    byte[] value = Bytes.toBytes(1);
+    try (Table table = TEST_UTIL.createTable(tableName, family)) {
+      table.put(new Put(row).addColumn(family, qual, value));
+    }
+    // change to FILE
+    admin.modifyTableStoreFileTracker(tableName, StoreFileTrackerFactory.Trackers.FILE.name())
+      .join();
+    verifyModifyTableResult(tableName, family, qual, row, value,
+      StoreFileTrackerFactory.Trackers.FILE.name());
+
+    // change to FILE again, should have no effect
+    admin.modifyTableStoreFileTracker(tableName, StoreFileTrackerFactory.Trackers.FILE.name())
+      .join();
+    verifyModifyTableResult(tableName, family, qual, row, value,
+      StoreFileTrackerFactory.Trackers.FILE.name());
+
+    // change to MIGRATION, and then to FILE
+    admin.modifyTable(TableDescriptorBuilder.newBuilder(admin.getDescriptor(tableName).join())
+      .setValue(StoreFileTrackerFactory.TRACKER_IMPL,
+        StoreFileTrackerFactory.Trackers.MIGRATION.name())
+      .setValue(SRC_IMPL, StoreFileTrackerFactory.Trackers.FILE.name())
+      .setValue(DST_IMPL, StoreFileTrackerFactory.Trackers.DEFAULT.name()).build()).join();
+    admin.modifyTableStoreFileTracker(tableName, StoreFileTrackerFactory.Trackers.FILE.name())
+      .join();
+    verifyModifyTableResult(tableName, family, qual, row, value,
+      StoreFileTrackerFactory.Trackers.FILE.name());
+
+    // change to MIGRATION, and then to DEFAULT
+    admin.modifyTable(TableDescriptorBuilder.newBuilder(admin.getDescriptor(tableName).join())
+      .setValue(StoreFileTrackerFactory.TRACKER_IMPL,
+        StoreFileTrackerFactory.Trackers.MIGRATION.name())
+      .setValue(SRC_IMPL, StoreFileTrackerFactory.Trackers.FILE.name())
+      .setValue(DST_IMPL, StoreFileTrackerFactory.Trackers.DEFAULT.name()).build()).join();
+    admin.modifyTableStoreFileTracker(tableName, StoreFileTrackerFactory.Trackers.DEFAULT.name())
+      .join();
+    verifyModifyTableResult(tableName, family, qual, row, value,
+      StoreFileTrackerFactory.Trackers.DEFAULT.name());
+  }
+
+  private void verifyModifyColumnFamilyResult(TableName tableName, byte[] family, byte[] qual,
+    byte[] row, byte[] value, String sft) throws IOException {
+    TableDescriptor td = admin.getDescriptor(tableName).join();
+    ColumnFamilyDescriptor cfd = td.getColumnFamily(family);
+    assertEquals(sft, cfd.getConfigurationValue(StoreFileTrackerFactory.TRACKER_IMPL));
+    // no migration related configs
+    assertNull(cfd.getConfigurationValue(SRC_IMPL));
+    assertNull(cfd.getConfigurationValue(DST_IMPL));
+    assertNull(cfd.getValue(SRC_IMPL));
+    assertNull(cfd.getValue(DST_IMPL));
+    try (Table table = TEST_UTIL.getConnection().getTable(tableName)) {
+      assertArrayEquals(value, table.get(new Get(row)).getValue(family, qual));
+    }
+  }
+
+  @Test
+  public void testModifyColumnFamilyStoreFileTracker() throws IOException {
+    byte[] family = Bytes.toBytes("info");
+    byte[] qual = Bytes.toBytes("q");
+    byte[] row = Bytes.toBytes(0);
+    byte[] value = Bytes.toBytes(1);
+    try (Table table = TEST_UTIL.createTable(tableName, family)) {
+      table.put(new Put(row).addColumn(family, qual, value));
+    }
+    // change to FILE
+    admin.modifyColumnFamilyStoreFileTracker(tableName, family,
+      StoreFileTrackerFactory.Trackers.FILE.name()).join();
+    verifyModifyColumnFamilyResult(tableName, family, qual, row, value,
+      StoreFileTrackerFactory.Trackers.FILE.name());
+
+    // change to FILE again, should have no effect
+    admin.modifyColumnFamilyStoreFileTracker(tableName, family,
+      StoreFileTrackerFactory.Trackers.FILE.name()).join();
+    verifyModifyColumnFamilyResult(tableName, family, qual, row, value,
+      StoreFileTrackerFactory.Trackers.FILE.name());
+
+    // change to MIGRATION, and then to FILE
+    TableDescriptor current = admin.getDescriptor(tableName).join();
+    admin.modifyTable(TableDescriptorBuilder.newBuilder(current)
+      .modifyColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(current.getColumnFamily(family))
+        .setConfiguration(StoreFileTrackerFactory.TRACKER_IMPL,
+          StoreFileTrackerFactory.Trackers.MIGRATION.name())
+        .setConfiguration(SRC_IMPL, StoreFileTrackerFactory.Trackers.FILE.name())
+        .setConfiguration(DST_IMPL, StoreFileTrackerFactory.Trackers.DEFAULT.name()).build())
+      .build()).join();
+    admin.modifyColumnFamilyStoreFileTracker(tableName, family,
+      StoreFileTrackerFactory.Trackers.FILE.name()).join();
+    verifyModifyColumnFamilyResult(tableName, family, qual, row, value,
+      StoreFileTrackerFactory.Trackers.FILE.name());
+
+    // change to MIGRATION, and then to DEFAULT
+    current = admin.getDescriptor(tableName).join();
+    admin.modifyTable(TableDescriptorBuilder.newBuilder(current)
+      .modifyColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(current.getColumnFamily(family))
+        .setConfiguration(StoreFileTrackerFactory.TRACKER_IMPL,
+          StoreFileTrackerFactory.Trackers.MIGRATION.name())
+        .setConfiguration(SRC_IMPL, StoreFileTrackerFactory.Trackers.FILE.name())
+        .setConfiguration(DST_IMPL, StoreFileTrackerFactory.Trackers.DEFAULT.name()).build())
+      .build()).join();
+    admin.modifyColumnFamilyStoreFileTracker(tableName, family,
+      StoreFileTrackerFactory.Trackers.DEFAULT.name()).join();
+    verifyModifyColumnFamilyResult(tableName, family, qual, row, value,
+      StoreFileTrackerFactory.Trackers.DEFAULT.name());
+  }
+
+  @Test
+  public void testModifyStoreFileTrackerError() throws IOException {
+    byte[] family = Bytes.toBytes("info");
+    TEST_UTIL.createTable(tableName, family).close();
+
+    // table not exists
+    assertThrows(TableNotFoundException.class,
+      () -> FutureUtils.get(admin.modifyTableStoreFileTracker(TableName.valueOf("whatever"),
+        StoreFileTrackerFactory.Trackers.FILE.name())));
+    // family not exists
+    assertThrows(NoSuchColumnFamilyException.class,
+      () -> FutureUtils.get(admin.modifyColumnFamilyStoreFileTracker(tableName,
+        Bytes.toBytes("not_exists"), StoreFileTrackerFactory.Trackers.FILE.name())));
+    // to migration
+    assertThrows(DoNotRetryIOException.class, () -> FutureUtils.get(admin
+      .modifyTableStoreFileTracker(tableName, StoreFileTrackerFactory.Trackers.MIGRATION.name())));
+    // disabled
+    admin.disableTable(tableName).join();
+    assertThrows(TableNotEnabledException.class, () -> FutureUtils.get(
+      admin.modifyTableStoreFileTracker(tableName, StoreFileTrackerFactory.Trackers.FILE.name())));
+  }
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
index 933addf..d53cf81 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
@@ -520,4 +520,16 @@ public class MockNoopMasterServices implements MasterServices {
   public MetaLocationSyncer getMetaLocationSyncer() {
     return null;
   }
+
+  @Override
+  public long modifyTableStoreFileTracker(TableName tableName, String dstSFT, long nonceGroup,
+    long nonce) throws IOException {
+    return -1;
+  }
+
+  @Override
+  public long modifyColumnStoreFileTracker(TableName tableName, byte[] family, String dstSFT,
+    long nonceGroup, long nonce) throws IOException {
+    return -1;
+  }
 }
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 91038e9..41f2afd 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,16 +22,9 @@ 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;
@@ -62,49 +55,4 @@ 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-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTrackerFactory.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTrackerValidationUtils.java
similarity index 69%
copy from hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTrackerFactory.java
copy to hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/storefiletracker/TestStoreFileTrackerValidationUtils.java
index 91038e9..a686b55 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/TestStoreFileTrackerValidationUtils.java
@@ -21,13 +21,11 @@ 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;
@@ -37,46 +35,26 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 @Category({ RegionServerTests.class, SmallTests.class })
-public class TestStoreFileTrackerFactory {
+public class TestStoreFileTrackerValidationUtils {
 
   @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()));
-  }
+    HBaseClassTestRule.forClass(TestStoreFileTrackerValidationUtils.class);
 
   @Test
   public void testCheckSFTCompatibility() throws Exception {
-    //checking default value change on different configuration levels
+    // 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
+    // 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
+    // creating a TD with matching ColumnFamilyDescriptor level setting
     TableDescriptorBuilder snapBuilder =
       TableDescriptorBuilder.newBuilder(TableName.valueOf("TableY"));
     snapBuilder.setValue(StoreFileTrackerFactory.TRACKER_IMPL, "FILE");
@@ -87,9 +65,9 @@ public class TestStoreFileTrackerFactory {
     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);
+    StoreFileTrackerValidationUtils.validatePreRestoreSnapshot(td, snapTd, conf);
     // removing cf level config is fine when it matches the td config
-    StoreFileTrackerFactory.validatePreRestoreSnapshot(snapTd, td, conf);
+    StoreFileTrackerValidationUtils.validatePreRestoreSnapshot(snapTd, td, conf);
 
     TableDescriptorBuilder defaultBuilder =
       TableDescriptorBuilder.newBuilder(TableName.valueOf("TableY"));
@@ -101,10 +79,10 @@ public class TestStoreFileTrackerFactory {
     TableDescriptor defaultTd = defaultBuilder.build();
 
     assertThrows(RestoreSnapshotException.class, () -> {
-      StoreFileTrackerFactory.validatePreRestoreSnapshot(td, defaultTd, conf);
+      StoreFileTrackerValidationUtils.validatePreRestoreSnapshot(td, defaultTd, conf);
     });
     assertThrows(RestoreSnapshotException.class, () -> {
-      StoreFileTrackerFactory.validatePreRestoreSnapshot(snapTd, defaultTd, conf);
+      StoreFileTrackerValidationUtils.validatePreRestoreSnapshot(snapTd, defaultTd, conf);
     });
   }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdmin.java
index 8b1a388..44a42f1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdmin.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdmin.java
@@ -24,7 +24,6 @@ import java.io.ByteArrayInputStream;
 import java.io.Closeable;
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.EnumSet;
 import java.util.List;
 import java.util.Map;
@@ -51,15 +50,13 @@ import org.apache.hadoop.hbase.client.CompactType;
 import org.apache.hadoop.hbase.client.CompactionState;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.client.ServerType;
 import org.apache.hadoop.hbase.client.LogEntry;
 import org.apache.hadoop.hbase.client.NormalizeTableFilterParams;
-import org.apache.hadoop.hbase.client.OnlineLogRecord;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.LogQueryFilter;
+import org.apache.hadoop.hbase.client.ServerType;
 import org.apache.hadoop.hbase.client.SnapshotDescription;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.TableDescriptor;
@@ -847,9 +844,8 @@ public class VerifyingRSGroupAdmin implements Admin, Closeable {
 
   @Override
   public List<LogEntry> getLogEntries(Set<ServerName> serverNames, String logType,
-      ServerType serverType, int limit, Map<String, Object> filterParams)
-      throws IOException {
-    return Collections.emptyList();
+    ServerType serverType, int limit, Map<String, Object> filterParams) throws IOException {
+    return admin.getLogEntries(serverNames, logType, serverType, limit, filterParams);
   }
 
   private void verify() throws IOException {
@@ -922,13 +918,19 @@ public class VerifyingRSGroupAdmin implements Admin, Closeable {
   }
 
   @Override
-  public List<OnlineLogRecord> getSlowLogResponses(Set<ServerName> serverNames,
-      LogQueryFilter logQueryFilter) throws IOException {
-    return null;
+  public List<Boolean> clearSlowLogResponses(Set<ServerName> serverNames) throws IOException {
+    return admin.clearSlowLogResponses(serverNames);
   }
 
   @Override
-  public List<Boolean> clearSlowLogResponses(Set<ServerName> serverNames) throws IOException {
-    return null;
+  public Future<Void> modifyColumnFamilyStoreFileTrackerAsync(TableName tableName, byte[] family,
+    String dstSFT) throws IOException {
+    return admin.modifyColumnFamilyStoreFileTrackerAsync(tableName, family, dstSFT);
+  }
+
+  @Override
+  public Future<Void> modifyTableStoreFileTrackerAsync(TableName tableName, String dstSFT)
+    throws IOException {
+    return admin.modifyTableStoreFileTrackerAsync(tableName, dstSFT);
   }
 }
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 71d3c9e..437bfca 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
@@ -1297,4 +1297,18 @@ public class ThriftAdmin implements Admin {
       throws IOException {
     throw new NotImplementedException("getLogEntries not supported in ThriftAdmin");
   }
+
+  @Override
+  public Future<Void> modifyColumnFamilyStoreFileTrackerAsync(TableName tableName, byte[] family,
+    String dstSFT) throws IOException {
+    throw new NotImplementedException(
+      "modifyColumnFamilyStoreFileTrackerAsync not supported in ThriftAdmin");
+  }
+
+  @Override
+  public Future<Void> modifyTableStoreFileTrackerAsync(TableName tableName, String dstSFT)
+    throws IOException {
+    throw new NotImplementedException(
+      "modifyTableStoreFileTrackerAsync not supported in ThriftAdmin");
+  }
 }