You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by bu...@apache.org on 2020/02/15 20:29:37 UTC

[hbase] 03/18: HBASE-22749: Distributed MOB compactions (some comments addressed)

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

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

commit 5827c4287c2e024474b94e0c1e8dca41ede18147
Author: Vladimir Rodionov <vr...@apache.org>
AuthorDate: Mon Jan 13 12:14:25 2020 -0800

    HBASE-22749: Distributed MOB compactions (some comments addressed)
---
 .../hadoop/hbase/IntegrationTestMobCompaction.java |  16 ++-
 .../org/apache/hadoop/hbase/master/HMaster.java    |  14 +--
 .../hadoop/hbase/master/MasterRpcServices.java     |   4 +-
 .../org/apache/hadoop/hbase/mob/MobConstants.java  |   5 +
 .../hbase/{master => mob}/MobFileCleanerChore.java | 116 ++++++++++++---------
 .../{master => mob}/MobFileCompactionChore.java    |   4 +-
 .../hadoop/hbase/mob/TesMobFileCleanerChore.java   |   1 -
 .../apache/hadoop/hbase/mob/TestMobCompaction.java |   1 -
 .../hadoop/hbase/mob/TestMobCompactionBase.java    |   1 -
 .../mob/TestMobCompactionOptRegionBatchMode.java   |   1 -
 .../TestMobCompactionRegularRegionBatchMode.java   |   1 -
 11 files changed, 91 insertions(+), 73 deletions(-)

diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMobCompaction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMobCompaction.java
index 47400a4..0ee1e09 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMobCompaction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMobCompaction.java
@@ -34,10 +34,10 @@ import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.master.MobFileCleanerChore;
 import org.apache.hadoop.hbase.master.cleaner.TimeToLiveHFileCleaner;
 import org.apache.hadoop.hbase.mob.FaultyMobStoreCompactor;
 import org.apache.hadoop.hbase.mob.MobConstants;
+import org.apache.hadoop.hbase.mob.MobFileCleanerChore;
 import org.apache.hadoop.hbase.mob.MobStoreEngine;
 import org.apache.hadoop.hbase.mob.MobUtils;
 
@@ -63,6 +63,13 @@ import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
  * FaultyMobStoreCompactor. The probability of failure is controlled by command-line
  * argument 'failprob'.
  * @see <a href="https://issues.apache.org/jira/browse/HBASE-22749">HBASE-22749</a>
+ * <p>
+ * Sample usage:
+ *
+ * <pre>
+ * hbase org.apache.hadoop.hbase.IntegrationTestMobCompaction -Dservers=10 -Drows=1000000
+ * -Dfailprob=0.2
+ * </pre>
  */
 @SuppressWarnings("deprecation")
 
@@ -142,6 +149,9 @@ public class IntegrationTestMobCompaction extends IntegrationTestBase {
   @Override
   public void setUpMonkey() throws Exception {
     // Sorry, no Monkey
+    String msg = "Chaos monkey is not supported";
+    LOG.warn(msg);
+    throw new IOException(msg);
   }
 
   private void deleteTablesIfAny() throws IOException {
@@ -400,10 +410,6 @@ public class IntegrationTestMobCompaction extends IntegrationTestBase {
     }
   }
 
-  /**
-   *
-   * @param args argument list
-   */
   public static void main(String[] args) throws Exception {
     Configuration conf = HBaseConfiguration.create();
     initConf(conf);
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 736eb6e..32d2fdc 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
@@ -148,6 +148,8 @@ import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.master.zksyncer.MasterAddressSyncer;
 import org.apache.hadoop.hbase.master.zksyncer.MetaLocationSyncer;
 import org.apache.hadoop.hbase.mob.MobConstants;
+import org.apache.hadoop.hbase.mob.MobFileCleanerChore;
+import org.apache.hadoop.hbase.mob.MobFileCompactionChore;
 import org.apache.hadoop.hbase.monitoring.MemoryBoundedLogMessageBuffer;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
@@ -1266,16 +1268,8 @@ public class HMaster extends HRegionServer implements MasterServices {
   private void initMobCleaner() {
     this.mobFileCleanerChore = new MobFileCleanerChore(this);
     getChoreService().scheduleChore(mobFileCleanerChore);
-
-    int mobCompactionPeriod = conf.getInt(MobConstants.MOB_COMPACTION_CHORE_PERIOD,
-      MobConstants.DEFAULT_MOB_COMPACTION_CHORE_PERIOD);
-
-    if (mobCompactionPeriod > 0) {
-      this.mobFileCompactionChore = new MobFileCompactionChore(this);
-      getChoreService().scheduleChore(mobFileCompactionChore);
-    } else {
-      LOG.info("The period is " + mobCompactionPeriod + " seconds, MobCompactionChore is disabled");
-    }
+    this.mobFileCompactionChore = new MobFileCompactionChore(this);
+    getChoreService().scheduleChore(mobFileCompactionChore);
   }
 
   /**
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 c1541fbf..fb19fe17 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
@@ -1762,7 +1762,9 @@ public class MasterRpcServices extends RSRpcServices
       if (MobUtils.isMobRegionName(tableName, regionName)) {
         checkHFileFormatVersionForMob();
         //TODO: support CompactType.MOB
-        LOG.warn("CompactType.MOB is not supported yet, will run regular compaction.");
+        // HBASE-23571
+        LOG.warn("CompactType.MOB is not supported yet, will run regular compaction."+
+            " Refer to HBASE-23571.");
         return super.compactRegion(controller, request);
       } else {
         return super.compactRegion(controller, request);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobConstants.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobConstants.java
index fa0944b..b2d54fb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobConstants.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobConstants.java
@@ -44,6 +44,11 @@ public final class MobConstants {
   public static final byte[] MOB_REGION_NAME_BYTES = Bytes.toBytes(MOB_REGION_NAME);
 
   public static final String MOB_CLEANER_PERIOD = "hbase.master.mob.cleaner.period";
+  
+  @Deprecated
+  public static final String DEPRECATED_MOB_CLEANER_PERIOD = 
+      "hbase.master.mob.ttl.cleaner.period";
+  
   public static final int DEFAULT_MOB_CLEANER_PERIOD = 24 * 60 * 60; // one day
 
   public static final String MOB_CACHE_EVICT_PERIOD = "hbase.mob.cache.evict.period";
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCleanerChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileCleanerChore.java
similarity index 75%
rename from hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCleanerChore.java
rename to hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileCleanerChore.java
index ca3df66..8594474 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCleanerChore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileCleanerChore.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.hbase.master;
+package org.apache.hadoop.hbase.mob;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
@@ -43,14 +43,11 @@ import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
-import org.apache.hadoop.hbase.mob.ExpiredMobFileCleaner;
-import org.apache.hadoop.hbase.mob.MobConstants;
-import org.apache.hadoop.hbase.mob.MobUtils;
+import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.regionserver.BloomType;
 import org.apache.hadoop.hbase.regionserver.HStoreFile;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.hadoop.hbase.util.HFileArchiveUtil;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -58,16 +55,21 @@ import org.slf4j.LoggerFactory;
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 
 /**
- * The Class ExpiredMobFileCleanerChore for running cleaner regularly to remove the expired
+ * The class MobFileCleanerChore for running cleaner regularly to remove the expired
  * and obsolete (files which have no active references to) mob files.
  */
+@SuppressWarnings("deprecation")
 @InterfaceAudience.Private
 public class MobFileCleanerChore extends ScheduledChore {
 
   private static final Logger LOG = LoggerFactory.getLogger(MobFileCleanerChore.class);
   private final HMaster master;
   private ExpiredMobFileCleaner cleaner;
-  private long minAgeToArchive;
+  
+  static {
+    Configuration.addDeprecation(MobConstants.DEPRECATED_MOB_CLEANER_PERIOD, 
+      MobConstants.MOB_CLEANER_PERIOD);
+  }
 
   public MobFileCleanerChore(HMaster master) {
     super(master.getServerName() + "-ExpiredMobFileCleanerChore", master,
@@ -84,9 +86,7 @@ public class MobFileCleanerChore extends ScheduledChore {
 
   private void checkObsoleteConfigurations() {
     Configuration conf = master.getConfiguration();
-    if (conf.get("hbase.master.mob.ttl.cleaner.period") != null) {
-      LOG.warn("'hbase.master.mob.ttl.cleaner.period' is obsolete and not used anymore.");
-    }
+
     if (conf.get("hbase.mob.compaction.mergeable.threshold") != null) {
       LOG.warn("'hbase.mob.compaction.mergeable.threshold' is obsolete and not used anymore.");
     }
@@ -111,23 +111,34 @@ public class MobFileCleanerChore extends ScheduledChore {
       justification = "Intentional")
 
   protected void chore() {
-    try {
+    TableDescriptors htds = master.getTableDescriptors();
 
-      TableDescriptors htds = master.getTableDescriptors();
-      Map<String, TableDescriptor> map = htds.getAll();
-      for (TableDescriptor htd : map.values()) {
-        for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) {
-          if (hcd.isMobEnabled() && hcd.getMinVersions() == 0) {
+    Map<String, TableDescriptor> map = null;
+    try {
+      map = htds.getAll();
+    } catch (IOException e) {
+      LOG.error("MobFileCleanerChore failed", e);
+      return;
+    }
+    for (TableDescriptor htd : map.values()) {
+      for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) {
+        if (hcd.isMobEnabled() && hcd.getMinVersions() == 0) {
+          try {
             cleaner.cleanExpiredMobFiles(htd.getTableName().getNameAsString(), hcd);
+          } catch (IOException e) {
+            LOG.error("Failed to clean the expired mob files table=" + htd.getTableName()
+                + " family=" + hcd.getNameAsString(), e);
           }
         }
+      }
+      try {
         // Now clean obsolete files for a table
-        LOG.info("Cleaning obsolete MOB files ...");
+        LOG.info("Cleaning obsolete MOB files from table={}", htd.getTableName());
         cleanupObsoleteMobFiles(master.getConfiguration(), htd.getTableName());
-        LOG.info("Cleaning obsolete MOB files finished");
+        LOG.info("Cleaning obsolete MOB files finished for table={}", htd.getTableName());
+      } catch (IOException e) {
+        LOG.error("Failed to clean the obsolete mob files for table=" + htd.getTableName(), e);
       }
-    } catch (Exception e) {
-      LOG.error("Fail to clean the expired mob files", e);
     }
   }
 
@@ -139,8 +150,17 @@ public class MobFileCleanerChore extends ScheduledChore {
    */
   public void cleanupObsoleteMobFiles(Configuration conf, TableName table) throws IOException {
 
-    this.minAgeToArchive =
+    long minAgeToArchive =
         conf.getLong(MobConstants.MIN_AGE_TO_ARCHIVE_KEY, MobConstants.DEFAULT_MIN_AGE_TO_ARCHIVE);
+    // We check only those MOB files, which creation time is less
+    // than maxCreationTimeToArchive. This is a current time - 1h. 1 hour gap
+    // gives us full confidence that all corresponding store files will
+    // exist at the time cleaning procedure begins and will be examined.
+    // So, if MOB file creation time is greater than this maxTimeToArchive,
+    // this will be skipped and won't be archived.
+    long maxCreationTimeToArchive = EnvironmentEdgeManager.currentTime() - minAgeToArchive;
+    LOG.info("Only MOB files whose creation time older than {} will be archived",
+      maxCreationTimeToArchive);
     try (final Connection conn = ConnectionFactory.createConnection(conf);
         final Admin admin = conn.getAdmin();) {
       TableDescriptor htd = admin.getDescriptor(table);
@@ -149,15 +169,7 @@ public class MobFileCleanerChore extends ScheduledChore {
         LOG.info("Skipping non-MOB table [{}]", table);
         return;
       }
-      // We check only those MOB files, which creation time is less
-      // than maxTimeToArchive. This is a current time - 1h. 1 hour gap
-      // gives us full confidence that all corresponding store files will
-      // exist at the time cleaning procedure begins and will be examined.
-      // So, if MOB file creation time is greater than this maxTimeToArchive,
-      // this will be skipped and won't be archived.
-      long maxCreationTimeToArchive = EnvironmentEdgeManager.currentTime() - minAgeToArchive;
-      LOG.info("Only MOB files whose creation time less than {} will be archived",
-        maxCreationTimeToArchive);
+
       Path rootDir = FSUtils.getRootDir(conf);
       Path tableDir = FSUtils.getTableDir(rootDir, table);
       // How safe is this call?
@@ -172,7 +184,11 @@ public class MobFileCleanerChore extends ScheduledChore {
           boolean succeed = false;
           Set<String> regionMobs = new HashSet<String>();
           while (!succeed) {
-
+            if (!fs.exists(storePath)) {
+              LOG.warn("Directory {} was deleted during cleaner procedure execution," 
+                  +" skipping.", storePath);
+              break;
+            }
             RemoteIterator<LocatedFileStatus> rit = fs.listLocatedStatus(storePath);
             List<Path> storeFiles = new ArrayList<Path>();
             // Load list of store files first
@@ -182,6 +198,7 @@ public class MobFileCleanerChore extends ScheduledChore {
                 storeFiles.add(p);
               }
             }
+            LOG.info("Found {} store files in: {}", storeFiles.size(), storePath);
             try {
               for (Path pp : storeFiles) {
                 LOG.debug("Store file: {}", pp);
@@ -198,7 +215,7 @@ public class MobFileCleanerChore extends ScheduledChore {
                     pp);
                   return;
                 } else if (mobRefData == null && bulkloadMarkerData != null) {
-                  LOG.info("Skipping file without MOB references (bulkloaded file):{}", pp);
+                  LOG.debug("Skipping file without MOB references (bulkloaded file):{}", pp);
                   continue;
                 }
                 // mobRefData will never be null here, but to make FindBugs happy
@@ -206,12 +223,16 @@ public class MobFileCleanerChore extends ScheduledChore {
                   // if length = 1 means NULL, that there are no MOB references
                   // in this store file, but the file was created by new MOB code
                   String[] mobs = new String(mobRefData).split(",");
-                  LOG.debug("Found: {} mob refs: ", mobs.length, Arrays.toString(mobs));
+                  if (LOG.isTraceEnabled()) {
+                    LOG.debug("Found: {} mob references: {}", mobs.length, Arrays.toString(mobs));
+                  } else {
+                    LOG.debug("Found: {} mob references", mobs.length);
+                  }
                   regionMobs.addAll(Arrays.asList(mobs));
                 }
               }
             } catch (FileNotFoundException e) {
-              LOG.warn("Starting MOB cleaning cycle from the beginning due to error:", e);
+              LOG.warn("Starting MOB cleaning cycle from the beginning due to error", e);
               continue;
             }
             succeed = true;
@@ -226,7 +247,7 @@ public class MobFileCleanerChore extends ScheduledChore {
           allActiveMobFileName.size());
       }
       LOG.debug("Found: {} active mob refs", allActiveMobFileName.size());
-      allActiveMobFileName.stream().forEach(LOG::debug);
+      allActiveMobFileName.stream().forEach(LOG::trace);
 
       // Now scan MOB directories and find MOB files with no references to them
       for (ColumnFamilyDescriptor hcd : list) {
@@ -242,19 +263,21 @@ public class MobFileCleanerChore extends ScheduledChore {
             // fresh, skip it in this case
             long creationTime = fs.getFileStatus(p).getModificationTime();
             if (creationTime < maxCreationTimeToArchive) {
-              LOG.info("Archiving MOB file {} creation time={}", p,
+              LOG.debug("Archiving MOB file {} creation time={}", p,
                 (fs.getFileStatus(p).getModificationTime()));
               toArchive.add(p);
             } else {
-              LOG.info("Skipping fresh file: {}", p);
+              LOG.debug("Skipping fresh file: {}", p);
             }
           } else {
-            LOG.info("Keepeing active MOB file: {}", p);
+            LOG.debug("Keeping active MOB file: {}", p);
           }
         }
-        LOG.info(" MOB Cleaner found {} files for family={}", toArchive.size(), family);
+        LOG.info(" MOB Cleaner found {} files to archive for table={} family={}", 
+          toArchive.size(), table, family);
         removeMobFiles(conf, table, family.getBytes(), toArchive);
-        LOG.info(" MOB Cleaner archived {} files", toArchive.size());
+        LOG.info(" MOB Cleaner archived {} files, table={} family={}", 
+          toArchive.size(), table, family);
       }
     }
   }
@@ -272,23 +295,16 @@ public class MobFileCleanerChore extends ScheduledChore {
 
     if (storeFiles.size() == 0) {
       // nothing to remove
-      LOG.debug("Skipping archiving old MOB file: collection is empty");
+      LOG.debug("Skipping archiving old MOB files - no files found.");
       return;
     }
     Path mobTableDir = FSUtils.getTableDir(MobUtils.getMobHome(conf), tableName);
     FileSystem fs = storeFiles.get(0).getFileSystem(conf);
-    Path storeArchiveDir = HFileArchiveUtil.getStoreArchivePath(conf,
-      MobUtils.getMobRegionInfo(tableName), mobTableDir, family);
 
     for (Path p : storeFiles) {
-      Path archiveFilePath = new Path(storeArchiveDir, p.getName());
-      if (fs.exists(archiveFilePath)) {
-        LOG.warn("MOB Cleaner skip archiving: {} because it has been archived already", p);
-        continue;
-      }
       LOG.info("MOB Cleaner is archiving: {}", p);
-      HFileArchiver.archiveStoreFile(conf, fs, MobUtils.getMobRegionInfo(tableName), mobTableDir,
-        family, p);
+      HFileArchiver.archiveStoreFile(conf, fs, MobUtils.getMobRegionInfo(tableName), 
+        mobTableDir, family, p);
     }
   }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCompactionChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileCompactionChore.java
similarity index 99%
rename from hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCompactionChore.java
rename to hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileCompactionChore.java
index 251abc8..2628013 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCompactionChore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileCompactionChore.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hbase.master;
+package org.apache.hadoop.hbase.mob;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -36,7 +36,7 @@ import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableState;
-import org.apache.hadoop.hbase.mob.MobConstants;
+import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TesMobFileCleanerChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TesMobFileCleanerChore.java
index deb1cd4..f13b824 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TesMobFileCleanerChore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TesMobFileCleanerChore.java
@@ -38,7 +38,6 @@ import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.master.MobFileCleanerChore;
 import org.apache.hadoop.hbase.master.cleaner.TimeToLiveHFileCleaner;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompaction.java
index 2b2cac3..e8e7357 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompaction.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompaction.java
@@ -41,7 +41,6 @@ import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.master.MobFileCleanerChore;
 import org.apache.hadoop.hbase.master.cleaner.TimeToLiveHFileCleaner;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Bytes;
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionBase.java
index 169c74c..50637da 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionBase.java
@@ -38,7 +38,6 @@ import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.master.MobFileCleanerChore;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.RegionSplitter;
 import org.junit.After;
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptRegionBatchMode.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptRegionBatchMode.java
index ba5e47e..7496f8c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptRegionBatchMode.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptRegionBatchMode.java
@@ -23,7 +23,6 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.master.MobFileCompactionChore;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.junit.Before;
 import org.junit.ClassRule;
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularRegionBatchMode.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularRegionBatchMode.java
index 936173c..f6c984a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularRegionBatchMode.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularRegionBatchMode.java
@@ -23,7 +23,6 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.master.MobFileCompactionChore;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.junit.Before;
 import org.junit.ClassRule;