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:35 UTC

[hbase] 01/18: HBASE-22749: Distributed MOB compactions

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 89b2e3a403826105eff8f98c267e631873106273
Author: Vladimir Rodionov <vr...@apache.org>
AuthorDate: Mon Dec 9 12:00:37 2019 -0800

    HBASE-22749: Distributed MOB compactions
---
 .../hadoop/hbase/IntegrationTestMobCompaction.java |  413 +++++++
 .../hadoop/hbase/io/hfile/HFileWriterImpl.java     |    4 +
 .../hbase/master/ExpiredMobFileCleanerChore.java   |   87 --
 .../org/apache/hadoop/hbase/master/HMaster.java    |   39 +-
 .../hbase/master/MasterMobCompactionThread.java    |  181 ---
 .../hadoop/hbase/master/MasterRpcServices.java     |   90 +-
 .../hadoop/hbase/master/MobCompactionChore.java    |   96 --
 .../hadoop/hbase/master/MobFileCleanerChore.java   |  289 +++++
 .../hbase/master/MobFileCompactionChore.java       |  222 ++++
 .../hadoop/hbase/mob/DefaultMobStoreCompactor.java |  459 ++++++--
 .../hadoop/hbase/mob/DefaultMobStoreFlusher.java   |   27 +-
 .../hadoop/hbase/mob/ExpiredMobFileCleaner.java    |    5 +-
 .../org/apache/hadoop/hbase/mob/MobConstants.java  |   79 +-
 .../org/apache/hadoop/hbase/mob/MobFileName.java   |   39 +-
 .../apache/hadoop/hbase/mob/MobStoreEngine.java    |   12 +-
 .../java/org/apache/hadoop/hbase/mob/MobUtils.java |  578 ++--------
 .../mob/compactions/MobCompactionRequest.java      |   64 --
 .../hadoop/hbase/mob/compactions/MobCompactor.java |   90 --
 .../PartitionedMobCompactionRequest.java           |  333 ------
 .../mob/compactions/PartitionedMobCompactor.java   |  949 ---------------
 .../hadoop/hbase/regionserver/HMobStore.java       |   42 +-
 .../apache/hadoop/hbase/regionserver/HStore.java   |    2 +-
 .../hadoop/hbase/regionserver/HStoreFile.java      |    6 +
 .../hadoop/hbase/regionserver/StoreFileInfo.java   |  114 +-
 .../hadoop/hbase/regionserver/StoreFileWriter.java |   29 +
 .../hbase/regionserver/compactions/Compactor.java  |    2 +-
 .../java/org/apache/hadoop/hbase/util/FSUtils.java |    2 +-
 .../hadoop/hbase/mob/FaultyMobStoreCompactor.java} |  349 +++---
 .../org/apache/hadoop/hbase/mob/MobStressTool.java |   79 ++
 .../hadoop/hbase/mob/TesMobFileCleanerChore.java   |  236 ++++
 .../apache/hadoop/hbase/mob/TestMobCompaction.java |  375 ++++++
 .../hadoop/hbase/mob/TestMobCompactionBase.java    |  242 ++++
 .../hadoop/hbase/mob/TestMobCompactionOptMode.java |   88 ++
 .../mob/TestMobCompactionOptRegionBatchMode.java   |   99 ++
 .../hbase/mob/TestMobCompactionRegularMode.java    |   80 ++
 .../TestMobCompactionRegularRegionBatchMode.java   |   96 ++
 .../apache/hadoop/hbase/mob/TestMobFileName.java   |   19 +-
 .../hbase/mob/compactions/TestMobCompactor.java    | 1214 --------------------
 .../TestPartitionedMobCompactionRequest.java       |   65 --
 .../compactions/TestPartitionedMobCompactor.java   |  961 ----------------
 .../regionserver/TestBulkLoadReplication.java      |   73 +-
 .../hbase/regionserver/TestMobStoreCompaction.java |   58 -
 .../hadoop/hbase/util/BaseTestHBaseFsck.java       |    2 +-
 .../apache/hadoop/hbase/util/TestHBaseFsckMOB.java |    1 +
 44 files changed, 3191 insertions(+), 5099 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
new file mode 100644
index 0000000..3373ce9
--- /dev/null
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMobCompaction.java
@@ -0,0 +1,413 @@
+/**
+ * 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;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Set;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.client.Admin;
+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.MobStoreEngine;
+import org.apache.hadoop.hbase.mob.MobUtils;
+
+import org.apache.hadoop.hbase.testclassification.IntegrationTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.hbase.thirdparty.com.google.common.base.MoreObjects;
+import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * An integration test to detect regressions in HBASE-22749. Test creates
+ * MOB-enabled table, and runs in parallel, the following tasks: loads data,
+ * runs MOB compactions, runs MOB cleaning chore. The failure injections into MOB
+ * compaction cycle is implemented via specific sub-class of DefaultMobStoreCompactor -
+ * 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>
+ */
+@SuppressWarnings("deprecation")
+
+@Category(IntegrationTests.class)
+public class IntegrationTestMobCompaction extends IntegrationTestBase {
+  protected static final Logger LOG = LoggerFactory.getLogger(IntegrationTestMobCompaction.class);
+
+  protected static final String REGIONSERVER_COUNT_KEY = "servers";
+  protected static final String ROWS_COUNT_KEY = "rows";
+  protected static final String FAILURE_PROB_KEY = "failprob";
+
+  protected static final int DEFAULT_REGIONSERVER_COUNT = 3;
+  protected static final int DEFAULT_ROWS_COUNT = 5000000;
+  protected static final double DEFAULT_FAILURE_PROB = 0.1;
+
+  protected static int regionServerCount = DEFAULT_REGIONSERVER_COUNT;
+  protected static long rowsToLoad = DEFAULT_ROWS_COUNT;
+  protected static double failureProb = DEFAULT_FAILURE_PROB;
+
+  protected static String famStr = "f1";
+  protected static byte[] fam = Bytes.toBytes(famStr);
+  protected static byte[] qualifier = Bytes.toBytes("q1");
+  protected static long mobLen = 10;
+  protected static byte[] mobVal = Bytes
+      .toBytes("01234567890123456789012345678901234567890123456789012345678901234567890123456789");
+
+  private static Configuration conf;
+  private static HTableDescriptor hdt;
+  private static HColumnDescriptor hcd;
+  private static Admin admin;
+  private static Table table = null;
+  private static MobFileCleanerChore chore;
+
+  private static volatile boolean run = true;
+
+  @Override
+  @Before
+  public void setUp() throws Exception {
+    util = getTestingUtil(getConf());
+    conf = util.getConfiguration();
+    // Initialize with test-specific configuration values
+    initConf(conf);
+    regionServerCount =
+        conf.getInt(REGIONSERVER_COUNT_KEY, DEFAULT_REGIONSERVER_COUNT);
+    LOG.info("Initializing cluster with {} region servers.", regionServerCount);
+    util.initializeCluster(regionServerCount);
+    admin = util.getAdmin();
+
+    createTestTable();
+
+    LOG.info("Cluster initialized and ready");
+  }
+
+  private void createTestTable() throws IOException {
+    // Create test table
+    hdt = util.createTableDescriptor("testMobCompactTable");
+    hcd = new HColumnDescriptor(fam);
+    hcd.setMobEnabled(true);
+    hcd.setMobThreshold(mobLen);
+    hcd.setMaxVersions(1);
+    hdt.addFamily(hcd);
+    table = util.createTable(hdt, null);
+  }
+
+  @After
+  public void tearDown() throws IOException {
+    LOG.info("Cleaning up after test.");
+    if(util.isDistributedCluster()) {
+      deleteTablesIfAny();
+      // TODO
+    }
+    LOG.info("Restoring cluster.");
+    util.restoreCluster();
+    LOG.info("Cluster restored.");
+  }
+
+  @Override
+  public void setUpMonkey() throws Exception {
+    // Sorry, no Monkey
+  }
+
+  private void deleteTablesIfAny() throws IOException {
+    if (table != null) {
+      util.deleteTableIfAny(table.getName());
+    }
+  }
+
+
+
+  @Override
+  public void setUpCluster() throws Exception {
+    util = getTestingUtil(getConf());
+    LOG.debug("Initializing/checking cluster has {} servers",regionServerCount);
+    util.initializeCluster(regionServerCount);
+    LOG.debug("Done initializing/checking cluster");
+  }
+
+  /**
+   *
+   * @return status of CLI execution
+   */
+  @Override
+  public int runTestFromCommandLine() throws Exception {
+    testMobCompaction();
+    return 0;
+  }
+
+  @Override
+  public TableName getTablename() {
+    // That is only valid when Monkey is CALM (no monkey)
+    return null;
+  }
+
+  @Override
+  protected Set<String> getColumnFamilies() {
+    // That is only valid when Monkey is CALM (no monkey)
+    return null;
+  }
+
+  @Override
+  protected void addOptions() {
+    addOptWithArg(REGIONSERVER_COUNT_KEY,
+      "Total number of region servers. Default: '" + DEFAULT_REGIONSERVER_COUNT + "'");
+    addOptWithArg(ROWS_COUNT_KEY,
+      "Total number of data rows to load. Default: '" + DEFAULT_ROWS_COUNT + "'");
+    addOptWithArg(FAILURE_PROB_KEY,
+      "Probability of a failure of a region MOB compaction request. Default: '"
+    + DEFAULT_FAILURE_PROB + "'");
+  }
+
+  @Override
+  protected void processOptions(CommandLine cmd) {
+    super.processOptions(cmd);
+
+    regionServerCount =
+        Integer.parseInt(cmd.getOptionValue(REGIONSERVER_COUNT_KEY,
+          Integer.toString(DEFAULT_REGIONSERVER_COUNT)));
+    rowsToLoad =
+        Long.parseLong(cmd.getOptionValue(ROWS_COUNT_KEY,
+          Long.toString(DEFAULT_ROWS_COUNT)));
+    failureProb = Double.parseDouble(cmd.getOptionValue(FAILURE_PROB_KEY,
+      Double.toString(DEFAULT_FAILURE_PROB)));
+
+    LOG.info(MoreObjects.toStringHelper("Parsed Options")
+      .add(REGIONSERVER_COUNT_KEY, regionServerCount)
+      .add(ROWS_COUNT_KEY, rowsToLoad)
+      .add(FAILURE_PROB_KEY, failureProb)
+      .toString());
+  }
+
+  private static void initConf(Configuration conf) {
+
+    conf.setInt("hfile.format.version", 3);
+    conf.setLong(TimeToLiveHFileCleaner.TTL_CONF_KEY, 0);
+    conf.setInt("hbase.client.retries.number", 100);
+    conf.setInt("hbase.hregion.max.filesize", 200000000);
+    conf.setInt("hbase.hregion.memstore.flush.size", 800000);
+    conf.setInt("hbase.hstore.blockingStoreFiles", 150);
+    conf.setInt("hbase.hstore.compaction.throughput.lower.bound", 52428800);
+    conf.setInt("hbase.hstore.compaction.throughput.higher.bound", 2 * 52428800);
+    conf.setDouble("injected.fault.probability", failureProb);
+    conf.set(MobStoreEngine.DEFAULT_MOB_COMPACTOR_CLASS_KEY,
+      FaultyMobStoreCompactor.class.getName());
+    conf.setBoolean("hbase.table.sanity.checks", false);
+    conf.setLong(MobConstants.MIN_AGE_TO_ARCHIVE_KEY, 20000);
+
+  }
+
+
+  class MajorCompaction implements Runnable {
+
+    @Override
+    public void run() {
+      while (run) {
+        try {
+          admin.majorCompact(hdt.getTableName(), fam);
+          Thread.sleep(120000);
+        } catch (Exception e) {
+          LOG.error("MOB Stress Test FAILED", e);
+          System.exit(-1);
+        }
+      }
+    }
+  }
+
+  class CleanMobAndArchive implements Runnable {
+
+    @Override
+    public void run() {
+      while (run) {
+        try {
+          LOG.info("MOB cleanup chore started ...");
+          if (chore == null) {
+            chore = new MobFileCleanerChore();
+          }
+          chore.cleanupObsoleteMobFiles(conf, table.getName());
+          LOG.info("MOB cleanup chore finished");
+
+          Thread.sleep(130000);
+        } catch (Exception e) {
+          e.printStackTrace();
+        }
+      }
+    }
+  }
+
+  class WriteData implements Runnable {
+
+    private long rows = -1;
+
+    public WriteData(long rows) {
+      this.rows = rows;
+    }
+
+    @Override
+    public void run() {
+      try {
+
+        // BufferedMutator bm = admin.getConnection().getBufferedMutator(table.getName());
+        // Put Operation
+        for (int i = 0; i < rows; i++) {
+          Put p = new Put(Bytes.toBytes(i));
+          p.addColumn(fam, qualifier, mobVal);
+          table.put(p);
+
+          // bm.mutate(p);
+          if (i % 10000 == 0) {
+            LOG.info("LOADED=" + i);
+            try {
+              Thread.sleep(500);
+            } catch (InterruptedException ee) {
+
+            }
+          }
+          if (i % 100000 == 0) {
+            printStats(i);
+          }
+        }
+        // bm.flush();
+        admin.flush(table.getName());
+        run = false;
+      } catch (Exception e) {
+        LOG.error("MOB Stress Test FAILED", e);
+        System.exit(-1);
+      }
+    }
+  }
+
+  @Test
+  public void testMobCompaction() throws InterruptedException, IOException {
+
+    try {
+
+      Thread writeData = new Thread(new WriteData(rowsToLoad));
+      writeData.start();
+
+      Thread majorcompact = new Thread(new MajorCompaction());
+      majorcompact.start();
+
+      Thread cleaner = new Thread(new CleanMobAndArchive());
+      cleaner.start();
+
+      while (run) {
+        Thread.sleep(1000);
+      }
+
+      getNumberOfMobFiles(conf, table.getName(), new String(fam));
+      LOG.info("Waiting for write thread to finish ...");
+      writeData.join();
+      // Cleanup again
+      chore.cleanupObsoleteMobFiles(conf, table.getName());
+
+      if (util != null) {
+        LOG.info("Archive cleaner started ...");
+        // Call archive cleaner again
+        util.getMiniHBaseCluster().getMaster().getHFileCleaner().choreForTesting();
+        LOG.info("Archive cleaner finished");
+      }
+
+      scanTable();
+
+    } finally {
+
+      admin.disableTable(hdt.getTableName());
+      admin.deleteTable(hdt.getTableName());
+    }
+    LOG.info("MOB Stress Test finished OK");
+    printStats(rowsToLoad);
+
+  }
+
+  private  long getNumberOfMobFiles(Configuration conf, TableName tableName, String family)
+      throws IOException {
+    FileSystem fs = FileSystem.get(conf);
+    Path dir = MobUtils.getMobFamilyPath(conf, tableName, family);
+    FileStatus[] stat = fs.listStatus(dir);
+    for (FileStatus st : stat) {
+      LOG.debug("MOB Directory content: {}", st.getPath());
+    }
+    LOG.debug("MOB Directory content total files: {}", stat.length);
+
+    return stat.length;
+  }
+
+  public void printStats(long loaded) {
+    LOG.info("MOB Stress Test: loaded=" + loaded + " compactions="
+        + FaultyMobStoreCompactor.totalCompactions.get() + " major="
+        + FaultyMobStoreCompactor.totalMajorCompactions.get() + " mob="
+        + FaultyMobStoreCompactor.mobCounter.get() + " injected failures="
+        + FaultyMobStoreCompactor.totalFailures.get());
+  }
+
+  private void scanTable() {
+    try {
+
+      Result result;
+      ResultScanner scanner = table.getScanner(fam);
+      int counter = 0;
+      while ((result = scanner.next()) != null) {
+        assertTrue(Arrays.equals(result.getValue(fam, qualifier), mobVal));
+        if (counter % 10000 == 0) {
+          LOG.info("GET=" + counter);
+        }
+        counter++;
+      }
+      assertEquals(rowsToLoad, counter);
+    } catch (Exception e) {
+      e.printStackTrace();
+      LOG.error("MOB Stress Test FAILED");
+      if (util != null) {
+        assertTrue(false);
+      } else {
+        System.exit(-1);
+      }
+    }
+  }
+
+  /**
+   *
+   * @param args argument list
+   */
+  public static void main(String[] args) throws Exception {
+    Configuration conf = HBaseConfiguration.create();
+    initConf(conf);
+    IntegrationTestingUtility.setUseDistributedCluster(conf);
+    int status = ToolRunner.run(conf, new IntegrationTestMobCompaction(), args);
+    System.exit(status);
+  }
+}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java
index 604ac1f..464ea49 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java
@@ -217,6 +217,10 @@ public class HFileWriterImpl implements HFile.Writer {
     HFile.updateWriteLatency(System.currentTimeMillis() - startTime);
   }
 
+  public long getPos() throws IOException {
+    return outputStream.getPos();
+
+  }
   /**
    * Checks that the given Cell's key does not violate the key order.
    *
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ExpiredMobFileCleanerChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ExpiredMobFileCleanerChore.java
deleted file mode 100644
index d37a80a..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ExpiredMobFileCleanerChore.java
+++ /dev/null
@@ -1,87 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.master;
-
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.hadoop.hbase.ScheduledChore;
-import org.apache.hadoop.hbase.TableDescriptors;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
-import org.apache.hadoop.hbase.client.TableDescriptor;
-import org.apache.hadoop.hbase.master.locking.LockManager;
-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.procedure2.LockType;
-
-/**
- * The Class ExpiredMobFileCleanerChore for running cleaner regularly to remove the expired
- * mob files.
- */
-@InterfaceAudience.Private
-public class ExpiredMobFileCleanerChore extends ScheduledChore {
-
-  private static final Logger LOG = LoggerFactory.getLogger(ExpiredMobFileCleanerChore.class);
-  private final HMaster master;
-  private ExpiredMobFileCleaner cleaner;
-
-  public ExpiredMobFileCleanerChore(HMaster master) {
-    super(master.getServerName() + "-ExpiredMobFileCleanerChore", master, master.getConfiguration()
-      .getInt(MobConstants.MOB_CLEANER_PERIOD, MobConstants.DEFAULT_MOB_CLEANER_PERIOD), master
-      .getConfiguration().getInt(MobConstants.MOB_CLEANER_PERIOD,
-        MobConstants.DEFAULT_MOB_CLEANER_PERIOD), TimeUnit.SECONDS);
-    this.master = master;
-    cleaner = new ExpiredMobFileCleaner();
-    cleaner.setConf(master.getConfiguration());
-  }
-
-  @Override
-  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="REC_CATCH_EXCEPTION",
-    justification="Intentional")
-  protected void chore() {
-    try {
-      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) {
-            // clean only for mob-enabled column.
-            // obtain a read table lock before cleaning, synchronize with MobFileCompactionChore.
-            final LockManager.MasterLock lock = master.getLockManager().createMasterLock(
-                MobUtils.getTableLockName(htd.getTableName()), LockType.SHARED,
-                this.getClass().getSimpleName() + ": Cleaning expired mob files");
-            try {
-              lock.acquire();
-              cleaner.cleanExpiredMobFiles(htd.getTableName().getNameAsString(), hcd);
-            } finally {
-              lock.release();
-            }
-          }
-        }
-      }
-    } catch (Exception e) {
-      LOG.error("Fail to clean the expired mob files", e);
-    }
-  }
-
-}
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 22011d4..736eb6e 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
@@ -396,9 +396,8 @@ public class HMaster extends HRegionServer implements MasterServices {
   private LogCleaner logCleaner;
   private HFileCleaner hfileCleaner;
   private ReplicationBarrierCleaner replicationBarrierCleaner;
-  private ExpiredMobFileCleanerChore expiredMobFileCleanerChore;
-  private MobCompactionChore mobCompactChore;
-  private MasterMobCompactionThread mobCompactThread;
+  private MobFileCleanerChore mobFileCleanerChore;
+  private MobFileCompactionChore mobFileCompactionChore;
   // used to synchronize the mobCompactionStates
   private final IdLock mobCompactionLock = new IdLock();
   // save the information of mob compactions in tables.
@@ -1265,14 +1264,18 @@ public class HMaster extends HRegionServer implements MasterServices {
   }
 
   private void initMobCleaner() {
-    this.expiredMobFileCleanerChore = new ExpiredMobFileCleanerChore(this);
-    getChoreService().scheduleChore(expiredMobFileCleanerChore);
+    this.mobFileCleanerChore = new MobFileCleanerChore(this);
+    getChoreService().scheduleChore(mobFileCleanerChore);
 
     int mobCompactionPeriod = conf.getInt(MobConstants.MOB_COMPACTION_CHORE_PERIOD,
-        MobConstants.DEFAULT_MOB_COMPACTION_CHORE_PERIOD);
-    this.mobCompactChore = new MobCompactionChore(this, mobCompactionPeriod);
-    getChoreService().scheduleChore(mobCompactChore);
-    this.mobCompactThread = new MasterMobCompactionThread(this);
+      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");
+    }
   }
 
   /**
@@ -1475,9 +1478,7 @@ public class HMaster extends HRegionServer implements MasterServices {
       }
     }
     stopChores();
-    if (this.mobCompactThread != null) {
-      this.mobCompactThread.close();
-    }
+
     super.stopServiceThreads();
     if (cleanerPool != null) {
       cleanerPool.shutdownNow();
@@ -1600,8 +1601,8 @@ public class HMaster extends HRegionServer implements MasterServices {
   private void stopChores() {
     ChoreService choreService = getChoreService();
     if (choreService != null) {
-      choreService.cancelChore(this.expiredMobFileCleanerChore);
-      choreService.cancelChore(this.mobCompactChore);
+      choreService.cancelChore(this.mobFileCleanerChore);
+      choreService.cancelChore(this.mobFileCompactionChore);
       choreService.cancelChore(this.balancerChore);
       choreService.cancelChore(this.normalizerChore);
       choreService.cancelChore(this.clusterStatusChore);
@@ -3431,16 +3432,6 @@ public class HMaster extends HRegionServer implements MasterServices {
     }
   }
 
-  /**
-   * Requests mob compaction.
-   * @param tableName The table the compact.
-   * @param columns The compacted columns.
-   * @param allFiles Whether add all mob files into the compaction.
-   */
-  public void requestMobCompaction(TableName tableName,
-                                   List<ColumnFamilyDescriptor> columns, boolean allFiles) throws IOException {
-    mobCompactThread.requestMobCompaction(conf, getFileSystem(), tableName, columns, allFiles);
-  }
 
   /**
    * Queries the state of the {@link LoadBalancerTracker}. If the balancer is not initialized,
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMobCompactionThread.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMobCompactionThread.java
deleted file mode 100644
index 0779eea..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMobCompactionThread.java
+++ /dev/null
@@ -1,181 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.master;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.RejectedExecutionException;
-import java.util.concurrent.SynchronousQueue;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
-import org.apache.hadoop.hbase.master.locking.LockManager;
-import org.apache.hadoop.hbase.mob.MobUtils;
-import org.apache.hadoop.hbase.procedure2.LockType;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
-
-/**
- * The mob compaction thread used in {@link MasterRpcServices}
- */
-@InterfaceAudience.Private
-public class MasterMobCompactionThread {
-  static final Logger LOG = LoggerFactory.getLogger(MasterMobCompactionThread.class);
-  private final HMaster master;
-  private final Configuration conf;
-  private final ExecutorService mobCompactorPool;
-  private final ExecutorService masterMobPool;
-
-  public MasterMobCompactionThread(HMaster master) {
-    this.master = master;
-    this.conf = master.getConfiguration();
-    final String n = Thread.currentThread().getName();
-    // this pool is used to run the mob compaction
-    this.masterMobPool = new ThreadPoolExecutor(1, 2, 60,
-        TimeUnit.SECONDS, new SynchronousQueue<>(),
-        new ThreadFactoryBuilder().setDaemon(true)
-            .setNameFormat(n + "-MasterMobCompaction-" + EnvironmentEdgeManager.currentTime())
-            .build());
-    ((ThreadPoolExecutor) this.masterMobPool).allowCoreThreadTimeOut(true);
-    // this pool is used in the mob compaction to compact the mob files by partitions
-    // in parallel
-    this.mobCompactorPool = MobUtils
-      .createMobCompactorThreadPool(master.getConfiguration());
-  }
-
-  /**
-   * Requests mob compaction
-   * @param conf The Configuration
-   * @param fs The file system
-   * @param tableName The table the compact
-   * @param columns The column descriptors
-   * @param allFiles Whether add all mob files into the compaction.
-   */
-  public void requestMobCompaction(Configuration conf, FileSystem fs, TableName tableName,
-                                   List<ColumnFamilyDescriptor> columns, boolean allFiles) throws IOException {
-    master.reportMobCompactionStart(tableName);
-    try {
-      masterMobPool.execute(new CompactionRunner(fs, tableName, columns,
-        allFiles, mobCompactorPool));
-    } catch (RejectedExecutionException e) {
-      // in case the request is rejected by the pool
-      try {
-        master.reportMobCompactionEnd(tableName);
-      } catch (IOException e1) {
-        LOG.error("Failed to mark end of mob compaction", e1);
-      }
-      throw e;
-    }
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("The mob compaction is requested for the columns " + columns
-        + " of the table " + tableName.getNameAsString());
-    }
-  }
-
-  private class CompactionRunner implements Runnable {
-    private FileSystem fs;
-    private TableName tableName;
-    private List<ColumnFamilyDescriptor> hcds;
-    private boolean allFiles;
-    private ExecutorService pool;
-
-    public CompactionRunner(FileSystem fs, TableName tableName, List<ColumnFamilyDescriptor> hcds,
-      boolean allFiles, ExecutorService pool) {
-      super();
-      this.fs = fs;
-      this.tableName = tableName;
-      this.hcds = hcds;
-      this.allFiles = allFiles;
-      this.pool = pool;
-    }
-
-    @Override
-    public void run() {
-      // These locks are on dummy table names, and only used for compaction/mob file cleaning.
-      final LockManager.MasterLock lock = master.getLockManager().createMasterLock(
-          MobUtils.getTableLockName(tableName), LockType.EXCLUSIVE,
-          this.getClass().getName() + ": mob compaction");
-      try {
-        for (ColumnFamilyDescriptor hcd : hcds) {
-          MobUtils.doMobCompaction(conf, fs, tableName, hcd, pool, allFiles, lock);
-        }
-      } catch (IOException e) {
-        LOG.error("Failed to perform the mob compaction", e);
-      } finally {
-        try {
-          master.reportMobCompactionEnd(tableName);
-        } catch (IOException e) {
-          LOG.error("Failed to mark end of mob compaction", e);
-        }
-      }
-    }
-  }
-
-  /**
-   * Only interrupt once it's done with a run through the work loop.
-   */
-  private void interruptIfNecessary() {
-    mobCompactorPool.shutdown();
-    masterMobPool.shutdown();
-  }
-
-  /**
-   * Wait for all the threads finish.
-   */
-  private void join() {
-    waitFor(mobCompactorPool, "Mob Compaction Thread");
-    waitFor(masterMobPool, "Region Server Mob Compaction Thread");
-  }
-
-  /**
-   * Closes the MasterMobCompactionThread.
-   */
-  public void close() {
-    interruptIfNecessary();
-    join();
-  }
-
-  /**
-   * Wait for thread finish.
-   * @param t the thread to wait
-   * @param name the thread name.
-   */
-  private void waitFor(ExecutorService t, String name) {
-    boolean done = false;
-    while (!done) {
-      try {
-        done = t.awaitTermination(60, TimeUnit.SECONDS);
-        LOG.info("Waiting for " + name + " to finish...");
-        if (!done) {
-          t.shutdownNow();
-        }
-      } catch (InterruptedException ie) {
-        LOG.warn("Interrupted waiting for " + name + " to finish...");
-      }
-    }
-  }
-}
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 5fb5f15..6d97f5d 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
@@ -32,6 +32,7 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
 import java.util.stream.Collectors;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.ClusterMetricsBuilder;
@@ -45,7 +46,6 @@ import org.apache.hadoop.hbase.ServerMetricsBuilder;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.UnknownRegionException;
-import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.MasterSwitchType;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.RegionInfo;
@@ -104,23 +104,6 @@ import org.apache.hadoop.hbase.security.access.PermissionStorage;
 import org.apache.hadoop.hbase.security.access.ShadedAccessControlUtil;
 import org.apache.hadoop.hbase.security.access.UserPermission;
 import org.apache.hadoop.hbase.security.visibility.VisibilityController;
-import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
-import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
-import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
-import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.zookeeper.KeeperException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
-import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
-import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
-
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos;
@@ -345,6 +328,21 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.Trans
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
+import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
+import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
+import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
+import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Implements the master RPC services.
@@ -1751,10 +1749,13 @@ public class MasterRpcServices extends RSRpcServices
       master.checkInitialized();
       byte[] regionName = request.getRegion().getValue().toByteArray();
       TableName tableName = RegionInfo.getTable(regionName);
+      // TODO: support CompactType.MOB
       // if the region is a mob region, do the mob file compaction.
       if (MobUtils.isMobRegionName(tableName, regionName)) {
         checkHFileFormatVersionForMob();
-        return compactMob(request, tableName);
+        //TODO: support CompactType.MOB
+        LOG.warn("CompactType.MOB is not supported yet, will run regular compaction.");
+        return super.compactRegion(controller, request);
       } else {
         return super.compactRegion(controller, request);
       }
@@ -1817,57 +1818,6 @@ public class MasterRpcServices extends RSRpcServices
     return builder.build();
   }
 
-  /**
-   * Compacts the mob files in the current table.
-   * @param request the request.
-   * @param tableName the current table name.
-   * @return The response of the mob file compaction.
-   * @throws IOException
-   */
-  private CompactRegionResponse compactMob(final CompactRegionRequest request,
-    TableName tableName) throws IOException {
-    if (!master.getTableStateManager().isTableState(tableName, TableState.State.ENABLED)) {
-      throw new DoNotRetryIOException("Table " + tableName + " is not enabled");
-    }
-    boolean allFiles = false;
-    List<ColumnFamilyDescriptor> compactedColumns = new ArrayList<>();
-    ColumnFamilyDescriptor[] hcds = master.getTableDescriptors().get(tableName).getColumnFamilies();
-    byte[] family = null;
-    if (request.hasFamily()) {
-      family = request.getFamily().toByteArray();
-      for (ColumnFamilyDescriptor hcd : hcds) {
-        if (Bytes.equals(family, hcd.getName())) {
-          if (!hcd.isMobEnabled()) {
-            LOG.error("Column family " + hcd.getNameAsString() + " is not a mob column family");
-            throw new DoNotRetryIOException("Column family " + hcd.getNameAsString()
-                    + " is not a mob column family");
-          }
-          compactedColumns.add(hcd);
-        }
-      }
-    } else {
-      for (ColumnFamilyDescriptor hcd : hcds) {
-        if (hcd.isMobEnabled()) {
-          compactedColumns.add(hcd);
-        }
-      }
-    }
-    if (compactedColumns.isEmpty()) {
-      LOG.error("No mob column families are assigned in the mob compaction");
-      throw new DoNotRetryIOException(
-              "No mob column families are assigned in the mob compaction");
-    }
-    if (request.hasMajor() && request.getMajor()) {
-      allFiles = true;
-    }
-    String familyLogMsg = (family != null) ? Bytes.toString(family) : "";
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("User-triggered mob compaction requested for table: "
-              + tableName.getNameAsString() + " for column family: " + familyLogMsg);
-    }
-    master.requestMobCompaction(tableName, compactedColumns, allFiles);
-    return CompactRegionResponse.newBuilder().build();
-  }
 
   @Override
   public IsBalancerEnabledResponse isBalancerEnabled(RpcController controller,
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobCompactionChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobCompactionChore.java
deleted file mode 100644
index 6c5d677..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobCompactionChore.java
+++ /dev/null
@@ -1,96 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.master;
-
-import java.util.Map;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.hadoop.hbase.ScheduledChore;
-import org.apache.hadoop.hbase.TableDescriptors;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
-import org.apache.hadoop.hbase.client.TableDescriptor;
-import org.apache.hadoop.hbase.client.TableState;
-import org.apache.hadoop.hbase.master.locking.LockManager;
-import org.apache.hadoop.hbase.mob.MobUtils;
-import org.apache.hadoop.hbase.procedure2.LockType;
-
-/**
- * The Class MobCompactChore for running compaction regularly to merge small mob files.
- */
-@InterfaceAudience.Private
-public class MobCompactionChore extends ScheduledChore {
-
-  private static final Logger LOG = LoggerFactory.getLogger(MobCompactionChore.class);
-  private HMaster master;
-  private ExecutorService pool;
-
-  public MobCompactionChore(HMaster master, int period) {
-    // use the period as initial delay.
-    super(master.getServerName() + "-MobCompactionChore", master, period, period, TimeUnit.SECONDS);
-    this.master = master;
-    this.pool = MobUtils.createMobCompactorThreadPool(master.getConfiguration());
-  }
-
-  @Override
-  protected void chore() {
-    try {
-      TableDescriptors htds = master.getTableDescriptors();
-      Map<String, TableDescriptor> map = htds.getAll();
-      for (TableDescriptor htd : map.values()) {
-        if (!master.getTableStateManager().isTableState(htd.getTableName(),
-          TableState.State.ENABLED)) {
-          continue;
-        }
-        boolean reported = false;
-        try {
-          final LockManager.MasterLock lock = master.getLockManager().createMasterLock(
-              MobUtils.getTableLockName(htd.getTableName()), LockType.EXCLUSIVE,
-              this.getClass().getName() + ": mob compaction");
-          for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) {
-            if (!hcd.isMobEnabled()) {
-              continue;
-            }
-            if (!reported) {
-              master.reportMobCompactionStart(htd.getTableName());
-              reported = true;
-            }
-            MobUtils.doMobCompaction(master.getConfiguration(), master.getFileSystem(),
-                htd.getTableName(), hcd, pool, false, lock);
-          }
-        } finally {
-          if (reported) {
-            master.reportMobCompactionEnd(htd.getTableName());
-          }
-        }
-      }
-    } catch (Exception e) {
-      LOG.error("Failed to compact mob files", e);
-    }
-  }
-
-  @Override
-  protected synchronized void cleanup() {
-    super.cleanup();
-    pool.shutdown();
-  }
-}
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/master/MobFileCleanerChore.java
new file mode 100644
index 0000000..0f1d0e1
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCleanerChore.java
@@ -0,0 +1,289 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.master;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.TableDescriptors;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.HFileArchiver;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+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.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.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The Class ExpiredMobFileCleanerChore for running cleaner regularly to remove the expired
+ * and obsolete (files which have no active references to) mob files.
+ */
+@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;
+
+  public MobFileCleanerChore(HMaster master) {
+    super(master.getServerName() + "-ExpiredMobFileCleanerChore", master, master.getConfiguration()
+      .getInt(MobConstants.MOB_CLEANER_PERIOD, MobConstants.DEFAULT_MOB_CLEANER_PERIOD), master
+      .getConfiguration().getInt(MobConstants.MOB_CLEANER_PERIOD,
+        MobConstants.DEFAULT_MOB_CLEANER_PERIOD), TimeUnit.SECONDS);
+    this.master = master;
+    cleaner = new ExpiredMobFileCleaner();
+    cleaner.setConf(master.getConfiguration());
+    checkObsoleteConfigurations();
+  }
+
+  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.");
+    }
+    if (conf.get("hbase.mob.delfile.max.count") != null) {
+      LOG.warn("'hbase.mob.delfile.max.count' is obsolete and not used anymore.");
+    }
+    if (conf.get("hbase.mob.compaction.threads.max") != null) {
+      LOG.warn("'hbase.mob.compaction.threads.max' is obsolete and not used anymore.");
+    }
+    if (conf.get("hbase.mob.compaction.batch.size") != null) {
+      LOG.warn("'hbase.mob.compaction.batch.size' is obsolete and not used anymore.");
+    }
+  }
+
+  @VisibleForTesting
+  public MobFileCleanerChore() {
+    this.master = null;
+  }
+
+  @Override
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="REC_CATCH_EXCEPTION",
+    justification="Intentional")
+
+  protected void chore() {
+    try {
+
+      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) {
+            cleaner.cleanExpiredMobFiles(htd.getTableName().getNameAsString(), hcd);
+          }
+        }
+        // Now clean obsolete files for a table
+        LOG.info("Cleaning obsolete MOB files ...");
+        cleanupObsoleteMobFiles(master.getConfiguration(), htd.getTableName());
+        LOG.info("Cleaning obsolete MOB files finished");
+      }
+    } catch (Exception e) {
+      LOG.error("Fail to clean the expired mob files", e);
+    }
+  }
+  /**
+   * Performs housekeeping file cleaning (called by MOB Cleaner chore)
+   * @param conf configuration
+   * @param table table name
+   * @throws IOException
+   */
+  public void cleanupObsoleteMobFiles(Configuration conf, TableName table)
+      throws IOException {
+
+    this.minAgeToArchive = conf.getLong(MobConstants.MIN_AGE_TO_ARCHIVE_KEY,
+      MobConstants.DEFAULT_MIN_AGE_TO_ARCHIVE);
+    try (final Connection conn = ConnectionFactory.createConnection(conf);
+        final Admin admin = conn.getAdmin();) {
+      TableDescriptor htd = admin.getDescriptor(table);
+      List<ColumnFamilyDescriptor> list = MobUtils.getMobColumnFamilies(htd);
+      if (list.size() == 0) {
+        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?
+      List<Path> regionDirs = FSUtils.getRegionDirs(FileSystem.get(conf), tableDir);
+
+      Set<String> allActiveMobFileName = new HashSet<String>();
+      FileSystem fs = FileSystem.get(conf);
+      for (Path regionPath : regionDirs) {
+        for (ColumnFamilyDescriptor hcd : list) {
+          String family = hcd.getNameAsString();
+          Path storePath = new Path(regionPath, family);
+          boolean succeed = false;
+          Set<String> regionMobs = new HashSet<String>();
+          while (!succeed) {
+
+            RemoteIterator<LocatedFileStatus> rit = fs.listLocatedStatus(storePath);
+            List<Path> storeFiles = new ArrayList<Path>();
+            // Load list of store files first
+            while (rit.hasNext()) {
+              Path p = rit.next().getPath();
+              if (fs.isFile(p)) {
+                storeFiles.add(p);
+              }
+            }
+            try {
+              for (Path pp : storeFiles) {
+                LOG.debug("Store file: {}", pp);
+                HStoreFile sf =
+                    new HStoreFile(fs, pp, conf, CacheConfig.DISABLED, BloomType.NONE, true);
+                sf.initReader();
+                byte[] mobRefData = sf.getMetadataValue(HStoreFile.MOB_FILE_REFS);
+                byte[] bulkloadMarkerData = sf.getMetadataValue(HStoreFile.BULKLOAD_TASK_KEY);
+                // close store file to avoid memory leaks
+                sf.closeStoreFile(true);
+                if (mobRefData == null && bulkloadMarkerData == null) {
+                  LOG.warn("Found old store file with no MOB_FILE_REFS: {} - "
+                    + "can not proceed until all old files will be MOB-compacted.", pp);
+                  return;
+                } else if (mobRefData == null && bulkloadMarkerData != null) {
+                  LOG.info("Skipping file without MOB references (bulkloaded file):{}", pp);
+                  continue;
+                }
+                if (mobRefData.length > 1) {
+                  // 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));
+                  regionMobs.addAll(Arrays.asList(mobs));
+                }
+              }
+            } catch (FileNotFoundException e) {
+              LOG.warn("Starting MOB cleaning cycle from the beginning due to error:",e);
+              continue;
+            }
+            succeed = true;
+          }
+          // Add MOB refs for current region/family
+          allActiveMobFileName.addAll(regionMobs);
+        } // END column families
+      } // END regions
+      // Check if number of MOB files too big (over 1M)
+      if (allActiveMobFileName.size() > 1000000) {
+        LOG.warn("Found too many active MOB files: {}, this may result in high memory pressure.",
+          allActiveMobFileName.size());
+      }
+      LOG.debug("Found: {} active mob refs", allActiveMobFileName.size());
+      allActiveMobFileName.stream().forEach(LOG::debug);
+
+      // Now scan MOB directories and find MOB files with no references to them
+      for (ColumnFamilyDescriptor hcd : list) {
+        List<Path> toArchive = new ArrayList<Path>();
+        String family = hcd.getNameAsString();
+        Path dir = MobUtils.getMobFamilyPath(conf, table, family);
+        RemoteIterator<LocatedFileStatus> rit = fs.listLocatedStatus(dir);
+        while (rit.hasNext()) {
+          LocatedFileStatus lfs = rit.next();
+          Path p = lfs.getPath();
+          if (!allActiveMobFileName.contains(p.getName())) {
+            // MOB is not in a list of active references, but it can be too
+            // fresh, skip it in this case
+            long creationTime = fs.getFileStatus(p).getModificationTime();
+            if ( creationTime < maxCreationTimeToArchive) {
+              /* DEBUG */ LOG.info(
+                " Archiving MOB file{} creation time=" + (fs.getFileStatus(p).getModificationTime()), p);
+              toArchive.add(p);
+            } else {
+              LOG.info("Skipping fresh file: {}", p);
+            }
+          } else {
+            LOG.info("Keepeing active MOB file: {}", p);
+          }
+        }
+        LOG.info(" MOB Cleaner found {} files for family={}", toArchive.size() , family);
+        removeMobFiles(conf, table, family.getBytes(), toArchive);
+        LOG.info(" MOB Cleaner archived {} files", toArchive.size());
+      }
+    }
+  }
+
+
+  /**
+   * Archives the mob files.
+   * @param conf The current configuration.
+   * @param tableName The table name.
+   * @param family The name of the column family.
+   * @param storeFiles The files to be archived.
+   * @throws IOException
+   */
+  public void removeMobFiles(Configuration conf, TableName tableName, byte[] family,
+      List<Path> storeFiles) throws IOException {
+
+    if (storeFiles.size() == 0) {
+      // nothing to remove
+      LOG.debug("Skipping archiving old MOB file: collection is empty");
+      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);
+    }
+  }
+}
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/master/MobFileCompactionChore.java
new file mode 100644
index 0000000..402718e
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCompactionChore.java
@@ -0,0 +1,222 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.TableDescriptors;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+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.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.util.EnvironmentEdgeManager;
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Periodic MOB compaction chore.
+ * It runs MOB compaction on region servers in parallel, thus
+ * utilizing distributed cluster resources. To avoid possible major
+ * compaction storms, one can specify maximum number regions to be compacted
+ * in parallel by setting configuration parameter: <br>
+ * 'hbase.mob.major.compaction.region.batch.size', which by default is 0 (unlimited).
+ *
+ */
+@InterfaceAudience.Private
+public class MobFileCompactionChore extends ScheduledChore {
+
+  private static final Logger LOG = LoggerFactory.getLogger(MobFileCompactionChore.class);
+  private Configuration conf;
+  private HMaster master;
+  private int regionBatchSize = 0;// not set - compact all
+
+  public MobFileCompactionChore(HMaster master) {
+    super(master.getServerName() + "-MobFileCompactionChore", master, master.getConfiguration()
+      .getInt(MobConstants.MOB_COMPACTION_CHORE_PERIOD,
+        MobConstants.DEFAULT_MOB_COMPACTION_CHORE_PERIOD), master
+      .getConfiguration().getInt(MobConstants.MOB_COMPACTION_CHORE_PERIOD,
+        MobConstants.DEFAULT_MOB_COMPACTION_CHORE_PERIOD), TimeUnit.SECONDS);
+    this.master = master;
+    this.conf = master.getConfiguration();
+    this.regionBatchSize =
+        master.getConfiguration().getInt(MobConstants.MOB_MAJOR_COMPACTION_REGION_BATCH_SIZE,
+          MobConstants.DEFAULT_MOB_MAJOR_COMPACTION_REGION_BATCH_SIZE);
+
+  }
+
+  @VisibleForTesting
+  public MobFileCompactionChore(Configuration conf, int batchSize) {
+    this.conf = conf;
+    this.regionBatchSize = batchSize;
+  }
+
+  @Override
+  protected void chore() {
+
+    boolean reported = false;
+
+    try (Connection conn = ConnectionFactory.createConnection(conf);
+         Admin admin = conn.getAdmin(); ) {
+
+      TableDescriptors htds = master.getTableDescriptors();
+      Map<String, TableDescriptor> map = htds.getAll();
+      for (TableDescriptor htd : map.values()) {
+        if (!master.getTableStateManager().isTableState(htd.getTableName(),
+          TableState.State.ENABLED)) {
+          LOG.debug("Skipping MOB compaction on table {} because it is not ENABLED",
+            htd.getTableName());
+          continue;
+        } else {
+          LOG.debug("Starting MOB compaction on table {}", htd.getTableName());
+        }
+        for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) {
+          try {
+            if (hcd.isMobEnabled()) {
+              if (!reported) {
+                master.reportMobCompactionStart(htd.getTableName());
+                reported = true;
+              }
+              LOG.info(" Major compacting {} cf={}", htd.getTableName(), hcd.getNameAsString());
+              if (regionBatchSize == MobConstants.DEFAULT_MOB_MAJOR_COMPACTION_REGION_BATCH_SIZE) {
+                LOG.debug("Batch compaction is disabled, {}=0", "hbase.mob.compaction.batch.size");
+                admin.majorCompact(htd.getTableName(), hcd.getName());
+              } else {
+                LOG.debug("Performing compaction in batches, {}={}",
+                  "hbase.mob.compaction.batch.size", regionBatchSize);
+                performMajorCompactionInBatches(admin, htd, hcd);
+              }
+            } else {
+              LOG.debug("Skipping column family {} because it is not MOB-enabled",
+                hcd.getNameAsString());
+            }
+          } catch (IOException e) {
+            LOG.error("Failed to compact table="+ htd.getTableName() +" cf="+ hcd.getNameAsString(),
+              e);
+          } catch (InterruptedException ee) {
+            Thread.currentThread().interrupt();
+            master.reportMobCompactionEnd(htd.getTableName());
+            LOG.warn("Failed to compact table="+ htd.getTableName() +" cf="+ hcd.getNameAsString(),
+              ee);
+            // Quit the chore
+            return;
+          }
+        }
+        if (reported) {
+          master.reportMobCompactionEnd(htd.getTableName());
+          reported = false;
+        }
+      }
+    } catch (IOException e) {
+      LOG.error("Failed to compact", e);
+    }
+  }
+
+  @VisibleForTesting
+  public void performMajorCompactionInBatches(Admin admin, TableDescriptor htd,
+      ColumnFamilyDescriptor hcd) throws IOException, InterruptedException {
+
+    List<RegionInfo> regions = admin.getRegions(htd.getTableName());
+    if (regions.size() <= this.regionBatchSize) {
+      LOG.debug("Performing compaction in non-batched mode, regions={}, batch size={}",
+        regions.size(), regionBatchSize);
+      admin.majorCompact(htd.getTableName(), hcd.getName());
+      return;
+    }
+    // Shuffle list of regions in case if they come ordered by region server
+    Collections.shuffle(regions);
+    // Create first batch
+    List<RegionInfo> toCompact = new ArrayList<RegionInfo>();
+    for (int i=0; i < this.regionBatchSize; i++) {
+      toCompact.add(regions.remove(0));
+    }
+
+    // Start compaction now
+    for(RegionInfo ri: toCompact) {
+      startCompaction(admin, htd.getTableName(), ri, hcd.getName());
+    }
+
+    List<RegionInfo> compacted = new ArrayList<RegionInfo>();
+    int totalCompacted = 0;
+    while(!toCompact.isEmpty()) {
+      // Check status of active compactions
+      for (RegionInfo ri: toCompact) {
+        try {
+          if (admin.getCompactionStateForRegion(ri.getRegionName()) == CompactionState.NONE) {
+            totalCompacted++;
+            LOG.info("Finished major compaction: table={} region={}, compacted regions={}",
+              htd.getTableName(),ri.getRegionNameAsString(), totalCompacted);
+            compacted.add(ri);
+          }
+        } catch (IOException e) {
+          LOG.warn("Could not get compaction state for region {}", ri.getEncodedName());
+        }
+      }
+      // Update batch: remove compacted regions and add new ones
+      for (RegionInfo ri: compacted) {
+        toCompact.remove(ri);
+        if (regions.size() > 0) {
+          RegionInfo region = regions.remove(0);
+          startCompaction(admin, htd.getTableName(),region, hcd.getName());
+          toCompact.add(region);
+        }
+      }
+      compacted.clear();
+      LOG.debug("Wait for 10 sec, toCompact size={} regions left={} compacted so far={}",
+        toCompact.size(), regions.size(), totalCompacted);
+      Thread.sleep(10000);
+    }
+    LOG.info("Finished major compacting {}. cf={}", htd.getTableName(), hcd.getNameAsString());
+
+  }
+
+  private void startCompaction(Admin admin, TableName table,  RegionInfo region, byte[] cf)
+      throws IOException, InterruptedException {
+
+    LOG.info("Started major compaction: table={} region={}", table,
+      region.getRegionNameAsString());
+    admin.majorCompactRegion(region.getRegionName());
+    // Wait until it really starts
+    // but with finite timeout
+    long waitTime = 300000; // 5 min
+    long startTime = EnvironmentEdgeManager.currentTime();
+    while(admin.getCompactionStateForRegion(region.getRegionName()) == CompactionState.NONE) {
+      // Is 1 second too aggressive?
+      Thread.sleep(1000);
+      if (EnvironmentEdgeManager.currentTime() - startTime > waitTime) {
+        LOG.warn("Waited for {} ms to start major compaction on table: {} region: {}. Aborted.",
+          waitTime, table.getNameAsString(), region.getRegionNameAsString());
+        break;
+      }
+    }
+  }
+}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java
index e73a7d2..d692723 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java
@@ -17,18 +17,27 @@
  */
 package org.apache.hadoop.hbase.mob;
 
+import static org.apache.hadoop.hbase.regionserver.ScanType.COMPACT_DROP_DELETES;
+import static org.apache.hadoop.hbase.regionserver.ScanType.COMPACT_RETAIN_DELETES;
+
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.util.ArrayList;
 import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Set;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.PrivateCellUtil;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.PrivateCellUtil;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.regionserver.CellSink;
 import org.apache.hadoop.hbase.regionserver.HMobStore;
 import org.apache.hadoop.hbase.regionserver.HStore;
@@ -48,6 +57,7 @@ import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -59,15 +69,52 @@ import org.slf4j.LoggerFactory;
 public class DefaultMobStoreCompactor extends DefaultCompactor {
 
   private static final Logger LOG = LoggerFactory.getLogger(DefaultMobStoreCompactor.class);
-  private long mobSizeThreshold;
-  private HMobStore mobStore;
+  protected long mobSizeThreshold;
+  protected HMobStore mobStore;
+
+  /*
+   *  MOB file reference set thread local variable. It contains set of
+   *  a MOB file names, which newly compacted store file has references to.
+   *  This variable is populated during compaction and the content of it is
+   *  written into meta section of a newly created store file at the final step
+   *  of compaction process.
+   */
+
+  static ThreadLocal<Set<String>> mobRefSet = new ThreadLocal<Set<String>>() {
+    @Override
+    protected Set<String> initialValue() {
+      return new HashSet<String>();
+    }
+  };
+
+  /*
+   * Is it user or system-originated request.
+   */
+
+  static ThreadLocal<Boolean> userRequest = new ThreadLocal<Boolean>() {
+    @Override
+    protected Boolean initialValue() {
+      return Boolean.FALSE;
+    }
+  };
+
+  /*
+   * Map : MOB file name - file length
+   * Can be expensive for large amount of MOB files?
+   */
+  static ThreadLocal<HashMap<String, Long>> mobLengthMap =
+      new ThreadLocal<HashMap<String, Long>>() {
+        @Override
+        protected HashMap<String, Long> initialValue() {
+          return new HashMap<String, Long>();
+        }
+      };
 
   private final InternalScannerFactory scannerFactory = new InternalScannerFactory() {
 
     @Override
     public ScanType getScanType(CompactionRequestImpl request) {
-      // retain the delete markers until they are expired.
-      return ScanType.COMPACT_RETAIN_DELETES;
+      return request.isAllFiles() ? COMPACT_DROP_DELETES : COMPACT_RETAIN_DELETES;
     }
 
     @Override
@@ -103,17 +150,59 @@ public class DefaultMobStoreCompactor extends DefaultCompactor {
     mobSizeThreshold = store.getColumnFamilyDescriptor().getMobThreshold();
   }
 
+
   @Override
-  public List<Path> compact(CompactionRequestImpl request, ThroughputController throughputController,
-      User user) throws IOException {
+  public List<Path> compact(CompactionRequestImpl request,
+      ThroughputController throughputController, User user) throws IOException {
+    LOG.info("Mob compaction: major=" + request.isMajor() + " isAll=" + request.isAllFiles()
+        + " priority=" + request.getPriority());
+    if (request.getPriority() == HStore.PRIORITY_USER) {
+      userRequest.set(Boolean.TRUE);
+    } else {
+      userRequest.set(Boolean.FALSE);
+    }
+    LOG.info("Mob compaction files: " + request.getFiles());
+    // Check if I/O optimized MOB compaction
+    if (conf.get(MobConstants.MOB_COMPACTION_TYPE_KEY, MobConstants.DEFAULT_MOB_COMPACTION_TYPE)
+        .equals(MobConstants.IO_OPTIMIZED_MOB_COMPACTION_TYPE)) {
+      if (request.isMajor() && request.getPriority() == HStore.PRIORITY_USER) {
+        Path mobDir = MobUtils.getMobFamilyPath(conf, store.getTableName(),
+          store.getColumnFamilyName());
+        List<Path> mobFiles = MobUtils.getReferencedMobFiles(request.getFiles(), mobDir);
+        if (mobFiles.size() > 0) {
+          calculateMobLengthMap(mobFiles);
+        }
+        LOG.info("I/O optimized MOB compaction. Total referenced MOB files: {}", mobFiles.size());
+      }
+    }
     return compact(request, scannerFactory, writerFactory, throughputController, user);
   }
 
+  private void calculateMobLengthMap(List<Path> mobFiles) throws IOException {
+    FileSystem fs = mobFiles.get(0).getFileSystem(this.conf);
+    HashMap<String, Long> map = mobLengthMap.get();
+    map.clear();
+    for (Path p: mobFiles) {
+      FileStatus st = fs.getFileStatus(p);
+      long size = st.getLen();
+      LOG.info("Ref MOB file={} size={}", p, size);
+      map.put(p.getName(), fs.getFileStatus(p).getLen());
+    }
+  }
+
+
   /**
    * Performs compaction on a column family with the mob flag enabled.
-   * This is for when the mob threshold size has changed or if the mob
-   * column family mode has been toggled via an alter table statement.
-   * Compacts the files by the following rules.
+   * This works only when MOB compaction is explicitly requested (by User), or by Master
+   * There are two modes of a MOB compaction:<br>
+   * <p>
+   * <ul>
+   * <li>1. Full mode - when all MOB data for a region is compacted into a single MOB file.
+   * <li>2. I/O optimized mode - for use cases with no or infrequent updates/deletes of a <br>
+   * MOB data. The main idea behind i/o optimized compaction is to limit maximum size of a MOB
+   * file produced during compaction and to limit I/O write/read amplification.
+   * </ul>
+   * The basic algorithm of compaction is the following: <br>
    * 1. If the Put cell has a mob reference tag, the cell's value is the path of the mob file.
    * <ol>
    * <li>
@@ -135,23 +224,6 @@ public class DefaultMobStoreCompactor extends DefaultCompactor {
    * Otherwise, directly write this cell into the store file.
    * </li>
    * </ol>
-   * 3. Decide how to write a Delete cell.
-   * <ol>
-   * <li>
-   * If a Delete cell does not have a mob reference tag which means this delete marker have not
-   * been written to the mob del file, write this cell to the mob del file, and write this cell
-   * with a ref tag to a store file.
-   * </li>
-   * <li>
-   * Otherwise, directly write it to a store file.
-   * </li>
-   * </ol>
-   * After the major compaction on the normal hfiles, we have a guarantee that we have purged all
-   * deleted or old version mob refs, and the delete markers are written to a del file with the
-   * suffix _del. Because of this, it is safe to use the del file in the mob compaction.
-   * The mob compaction doesn't take place in the normal hfiles, it occurs directly in the
-   * mob files. When the small mob files are merged into bigger ones, the del file is added into
-   * the scanner to filter the deleted cells.
    * @param fd File details
    * @param scanner Where to read from.
    * @param writer Where to write to.
@@ -169,6 +241,25 @@ public class DefaultMobStoreCompactor extends DefaultCompactor {
     long bytesWrittenProgressForCloseCheck = 0;
     long bytesWrittenProgressForLog = 0;
     long bytesWrittenProgressForShippedCall = 0;
+    // Clear old mob references
+    mobRefSet.get().clear();
+    boolean isUserRequest = userRequest.get();
+    boolean compactMOBs = major && isUserRequest;
+    boolean ioOptimizedMode = conf.get(MobConstants.MOB_COMPACTION_TYPE_KEY,
+      MobConstants.DEFAULT_MOB_COMPACTION_TYPE)
+        .equals(MobConstants.IO_OPTIMIZED_MOB_COMPACTION_TYPE);
+
+    boolean discardMobMiss =
+        conf.getBoolean(MobConstants.MOB_UNSAFE_DISCARD_MISS_KEY,
+          MobConstants.DEFAULT_MOB_DISCARD_MISS);
+
+    long maxMobFileSize = conf.getLong(MobConstants.MOB_COMPACTION_MAX_FILE_SIZE_KEY,
+      MobConstants.DEFAULT_MOB_COMPACTION_MAX_FILE_SIZE);
+    LOG.info("Compact MOB={} optimized={} maximum MOB file size={} major={}", compactMOBs,
+      ioOptimizedMode, maxMobFileSize, major);
+
+    FileSystem fs = FileSystem.get(conf);
+
     // Since scanner.next() can return 'false' but still be delivering data,
     // we have to use a do/while loop.
     List<Cell> cells = new ArrayList<>();
@@ -183,91 +274,163 @@ public class DefaultMobStoreCompactor extends DefaultCompactor {
     boolean hasMore;
     Path path = MobUtils.getMobFamilyPath(conf, store.getTableName(), store.getColumnFamilyName());
     byte[] fileName = null;
-    StoreFileWriter mobFileWriter = null, delFileWriter = null;
-    long mobCells = 0, deleteMarkersCount = 0;
+    StoreFileWriter mobFileWriter = null;
+    /*
+     * mobCells are used only to decide if we need to commit or abort current MOB output file.
+     */
+    long mobCells = 0;
     long cellsCountCompactedToMob = 0, cellsCountCompactedFromMob = 0;
     long cellsSizeCompactedToMob = 0, cellsSizeCompactedFromMob = 0;
     boolean finished = false;
+
     ScannerContext scannerContext =
         ScannerContext.newBuilder().setBatchLimit(compactionKVMax).build();
     throughputController.start(compactionName);
-    KeyValueScanner kvs = (scanner instanceof KeyValueScanner)? (KeyValueScanner)scanner : null;
-    long shippedCallSizeLimit = (long) numofFilesToCompact * this.store.getColumnFamilyDescriptor().getBlocksize();
+    KeyValueScanner kvs = (scanner instanceof KeyValueScanner) ? (KeyValueScanner) scanner : null;
+    long shippedCallSizeLimit =
+        (long) numofFilesToCompact * this.store.getColumnFamilyDescriptor().getBlocksize();
+
+    Cell mobCell = null;
     try {
-      try {
-        // If the mob file writer could not be created, directly write the cell to the store file.
-        mobFileWriter = mobStore.createWriterInTmp(new Date(fd.latestPutTs), fd.maxKeyCount,
-          compactionCompression, store.getRegionInfo().getStartKey(), true);
-        fileName = Bytes.toBytes(mobFileWriter.getPath().getName());
-      } catch (IOException e) {
-        LOG.warn("Failed to create mob writer, "
-               + "we will continue the compaction by writing MOB cells directly in store files", e);
-      }
-      if (major) {
-        try {
-          delFileWriter = mobStore.createDelFileWriterInTmp(new Date(fd.latestPutTs),
-            fd.maxKeyCount, compactionCompression, store.getRegionInfo().getStartKey());
-        } catch (IOException e) {
-          LOG.warn(
-            "Failed to create del writer, "
-            + "we will continue the compaction by writing delete markers directly in store files",
-            e);
-        }
-      }
+
+      mobFileWriter = newMobWriter(fd);
+      fileName = Bytes.toBytes(mobFileWriter.getPath().getName());
+
       do {
         hasMore = scanner.next(cells, scannerContext);
-        if (LOG.isDebugEnabled()) {
-          now = EnvironmentEdgeManager.currentTime();
-        }
+        now = EnvironmentEdgeManager.currentTime();
         for (Cell c : cells) {
-          if (major && CellUtil.isDelete(c)) {
-            if (MobUtils.isMobReferenceCell(c) || delFileWriter == null) {
-              // Directly write it to a store file
-              writer.append(c);
+          if (compactMOBs) {
+            if (MobUtils.isMobReferenceCell(c)) {
+              String fName = MobUtils.getMobFileName(c);
+              Path pp = new Path(new Path(fs.getUri()), new Path(path, fName));
+
+              // Added to support migration
+              try {
+                mobCell = mobStore.resolve(c, true, false).getCell();
+              } catch (FileNotFoundException fnfe) {
+                if (discardMobMiss) {
+                  LOG.debug("Missing MOB cell: file={} not found cell={}", pp, c);
+                  continue;
+                } else {
+                  throw fnfe;
+                }
+              }
+
+              if (discardMobMiss && mobCell.getValueLength() == 0) {
+                LOG.error("Missing MOB cell value: file=" + pp + " cell=" + mobCell);
+                continue;
+              } else if (mobCell.getValueLength() == 0) {
+                //TODO: what to do here? This is data corruption?
+                LOG.warn("Found 0 length MOB cell in a file={} cell={}", pp, mobCell);
+              }
+
+              if (mobCell.getValueLength() > mobSizeThreshold) {
+                // put the mob data back to the MOB store file
+                PrivateCellUtil.setSequenceId(mobCell, c.getSequenceId());
+                if (!ioOptimizedMode) {
+                  mobFileWriter.append(mobCell);
+                  mobCells++;
+                  writer.append(MobUtils.createMobRefCell(mobCell, fileName,
+                    this.mobStore.getRefCellTags()));
+                } else {
+                  // I/O optimized mode
+                  // Check if MOB cell origin file size is
+                  // greater than threshold
+                  Long size = mobLengthMap.get().get(fName);
+                  if (size == null) {
+                    // FATAL error, abort compaction
+                    String msg =
+                        String.format("Found unreferenced MOB file during compaction %s, aborting.",
+                      fName);
+                    LOG.error(msg);
+                    throw new IOException(msg);
+                  }
+                  // Can not be null
+                  if (size < maxMobFileSize) {
+                    // If MOB cell origin file is below threshold
+                    // it is get compacted
+                    mobFileWriter.append(mobCell);
+                    // Update number of mobCells in a current mob writer
+                    mobCells++;
+                    writer.append(MobUtils.createMobRefCell(mobCell, fileName,
+                      this.mobStore.getRefCellTags()));
+                    // Update total size of the output (we do not take into account
+                    // file compression yet)
+                    long len = getLength(mobFileWriter);
+
+                    if (len > maxMobFileSize) {
+                      LOG.debug("Closing output MOB File, length={} file={}",
+                        len, Bytes.toString(fileName));
+                      commitOrAbortMobWriter(mobFileWriter, fd.maxSeqId, mobCells, major);
+                      mobFileWriter = newMobWriter(fd);
+                      fileName = Bytes.toBytes(mobFileWriter.getPath().getName());
+                      mobCells = 0;
+                    }
+                  } else {
+                    // We leave large MOB file as is (is not compacted),
+                    // then we update set of MOB file references
+                    // and append mob cell directly to the store's writer
+                    mobRefSet.get().add(fName);
+                    writer.append(mobCell);
+                  }
+                }
+              } else {
+                // If MOB value is less than threshold, append it directly to a store file
+                PrivateCellUtil.setSequenceId(mobCell, c.getSequenceId());
+                writer.append(mobCell);
+                cellsCountCompactedFromMob++;
+                cellsSizeCompactedFromMob += mobCell.getValueLength();
+              }
             } else {
-              // Add a ref tag to this cell and write it to a store file.
-              writer.append(MobUtils.createMobRefDeleteMarker(c));
-              // Write the cell to a del file
-              delFileWriter.append(c);
-              deleteMarkersCount++;
+              // Not a MOB reference cell
+              int size = c.getValueLength();
+              if (size > mobSizeThreshold) {
+                // This MOB cell comes from a regular store file
+                // therefore we store it into original mob output
+                mobFileWriter.append(c);
+                writer
+                    .append(MobUtils.createMobRefCell(c, fileName, this.mobStore.getRefCellTags()));
+                mobCells++;
+                cellsCountCompactedToMob++;
+                cellsSizeCompactedToMob += c.getValueLength();
+                if (ioOptimizedMode) {
+                  // Update total size of the output (we do not take into account
+                  // file compression yet)
+                  long len = getLength(mobFileWriter);
+                  if (len > maxMobFileSize) {
+                    commitOrAbortMobWriter(mobFileWriter, fd.maxSeqId, mobCells, major);
+                    mobFileWriter = newMobWriter(fd);
+                    fileName = Bytes.toBytes(mobFileWriter.getPath().getName());
+                    mobCells = 0;
+                  }
+                }
+              } else {
+                // Not a MOB cell, write it directly to a store file
+                writer.append(c);
+              }
             }
-          } else if (mobFileWriter == null || c.getTypeByte() != KeyValue.Type.Put.getCode()) {
-            // If the mob file writer is null or the kv type is not put, directly write the cell
+          } else if (c.getTypeByte() != KeyValue.Type.Put.getCode()) {
+            // Not a major compaction or major with MOB disabled
+            // If the kv type is not put, directly write the cell
             // to the store file.
             writer.append(c);
           } else if (MobUtils.isMobReferenceCell(c)) {
+            // Not a major MOB compaction, Put MOB reference
             if (MobUtils.hasValidMobRefCellValue(c)) {
-              int size = MobUtils.getMobValueLength(c);
-              if (size > mobSizeThreshold) {
-                // If the value size is larger than the threshold, it's regarded as a mob. Since
-                // its value is already in the mob file, directly write this cell to the store file
-                writer.append(c);
-              } else {
-                // If the value is not larger than the threshold, it's not regarded a mob. Retrieve
-                // the mob cell from the mob file, and write it back to the store file. Must
-                // close the mob scanner once the life cycle finished.
-                try (MobCell mobCell = mobStore.resolve(c, false)) {
-                  if (mobCell.getCell().getValueLength() != 0) {
-                    // put the mob data back to the store file
-                    PrivateCellUtil.setSequenceId(mobCell.getCell(), c.getSequenceId());
-                    writer.append(mobCell.getCell());
-                    cellsCountCompactedFromMob++;
-                    cellsSizeCompactedFromMob += mobCell.getCell().getValueLength();
-                  } else {
-                    // If the value of a file is empty, there might be issues when retrieving,
-                    // directly write the cell to the store file, and leave it to be handled by the
-                    // next compaction.
-                    writer.append(c);
-                  }
-                }
-              }
+              // We do not check mobSizeThreshold during normal compaction,
+              // leaving it to a MOB compaction run
+              writer.append(c);
+              // Add MOB reference to a MOB reference set
+              mobRefSet.get().add(MobUtils.getMobFileName(c));
             } else {
-              LOG.warn("The value format of the KeyValue " + c
-                  + " is wrong, its length is less than " + Bytes.SIZEOF_INT);
+              // TODO ????
+              LOG.error("Corrupted MOB reference: " + c);
               writer.append(c);
             }
           } else if (c.getValueLength() <= mobSizeThreshold) {
-            //If value size of a cell is not larger than the threshold, directly write to store file
+            // If the value size of a cell is not larger than the threshold, directly write it to
+            // the store file.
             writer.append(c);
           } else {
             // If the value size of a cell is larger than the threshold, it's regarded as a mob,
@@ -275,13 +438,22 @@ public class DefaultMobStoreCompactor extends DefaultCompactor {
             mobCells++;
             // append the original keyValue in the mob file.
             mobFileWriter.append(c);
-            Cell reference = MobUtils.createMobRefCell(c, fileName,
-                this.mobStore.getRefCellTags());
+            Cell reference = MobUtils.createMobRefCell(c, fileName, this.mobStore.getRefCellTags());
             // write the cell whose value is the path of a mob file to the store file.
             writer.append(reference);
             cellsCountCompactedToMob++;
             cellsSizeCompactedToMob += c.getValueLength();
+            if (ioOptimizedMode) {
+              long len = getLength(mobFileWriter);
+              if (len > maxMobFileSize) {
+                commitOrAbortMobWriter(mobFileWriter, fd.maxSeqId, mobCells, major);
+                mobFileWriter = newMobWriter(fd);
+                fileName = Bytes.toBytes(mobFileWriter.getPath().getName());
+                mobCells = 0;
+              }
+            }
           }
+
           int len = c.getSerializedSize();
           ++progress.currentCompactedKVs;
           progress.totalCompactedSize += len;
@@ -302,7 +474,7 @@ public class DefaultMobStoreCompactor extends DefaultCompactor {
             }
           }
           if (kvs != null && bytesWrittenProgressForShippedCall > shippedCallSizeLimit) {
-            ((ShipperListener)writer).beforeShipped();
+            ((ShipperListener) writer).beforeShipped();
             kvs.shipped();
             bytesWrittenProgressForShippedCall = 0;
           }
@@ -326,6 +498,9 @@ public class DefaultMobStoreCompactor extends DefaultCompactor {
       progress.cancel();
       throw new InterruptedIOException(
           "Interrupted while control throughput of compacting " + compactionName);
+    } catch (IOException t) {
+      LOG.error("Mob compaction failed for region:{} ", store.getRegionInfo().getEncodedName());
+      throw t;
     } finally {
       // Clone last cell in the final because writer will append last cell when committing. If
       // don't clone here and once the scanner get closed, then the memory of last cell will be
@@ -333,40 +508,90 @@ public class DefaultMobStoreCompactor extends DefaultCompactor {
       ((ShipperListener) writer).beforeShipped();
       throughputController.finish(compactionName);
       if (!finished && mobFileWriter != null) {
+        // Remove all MOB references because compaction failed
+        mobRefSet.get().clear();
+        // Abort writer
+        LOG.debug("Aborting writer for {} because of a compaction failure",
+          mobFileWriter.getPath());
         abortWriter(mobFileWriter);
       }
-      if (!finished && delFileWriter != null) {
-        abortWriter(delFileWriter);
-      }
     }
-    if (delFileWriter != null) {
-      if (deleteMarkersCount > 0) {
-        // If the del file is not empty, commit it.
-        // If the commit fails, the compaction is re-performed again.
-        delFileWriter.appendMetadata(fd.maxSeqId, major, deleteMarkersCount);
-        delFileWriter.close();
-        mobStore.commitFile(delFileWriter.getPath(), path);
-      } else {
-        // If the del file is empty, delete it instead of committing.
-        abortWriter(delFileWriter);
-      }
+
+    // Commit last MOB writer
+    commitOrAbortMobWriter(mobFileWriter, fd.maxSeqId, mobCells, major);
+
+    mobStore.updateCellsCountCompactedFromMob(cellsCountCompactedFromMob);
+    mobStore.updateCellsCountCompactedToMob(cellsCountCompactedToMob);
+    mobStore.updateCellsSizeCompactedFromMob(cellsSizeCompactedFromMob);
+    mobStore.updateCellsSizeCompactedToMob(cellsSizeCompactedToMob);
+    progress.complete();
+    return true;
+  }
+
+  private long getLength(StoreFileWriter mobFileWriter) throws IOException {
+    return mobFileWriter.getPos();
+  }
+
+
+  private StoreFileWriter newMobWriter(FileDetails fd/*, boolean compactMOBs*/)
+      throws IOException {
+    try {
+      StoreFileWriter mobFileWriter = mobStore.createWriterInTmp(new Date(fd.latestPutTs),
+        fd.maxKeyCount, compactionCompression, store.getRegionInfo().getStartKey(), true);
+      LOG.debug("New MOB writer created={}", mobFileWriter.getPath().getName());
+      // Add reference we get for compact MOB
+      mobRefSet.get().add(mobFileWriter.getPath().getName());
+      return mobFileWriter;
+    } catch (IOException e) {
+      // Bailing out
+      LOG.error("Failed to create mob writer, ", e);
+      throw e;
     }
+  }
+
+  private void commitOrAbortMobWriter(StoreFileWriter mobFileWriter, long maxSeqId,
+     long mobCells, boolean major) throws IOException
+  {
+    // Commit or abort major mob writer
+    // If IOException happens during below operation, some
+    // MOB files can be committed partially, but corresponding
+    // store file won't be committed, therefore these MOB files
+    // become orphans and will be deleted during next MOB cleaning chore cycle
+    LOG.debug("Commit or abort size={} mobCells={} major={} file={}",
+      mobFileWriter.getPos(), mobCells, major, mobFileWriter.getPath().getName());
+    Path path = MobUtils.getMobFamilyPath(conf, store.getTableName(), store.getColumnFamilyName());
     if (mobFileWriter != null) {
       if (mobCells > 0) {
         // If the mob file is not empty, commit it.
-        mobFileWriter.appendMetadata(fd.maxSeqId, major, mobCells);
+        mobFileWriter.appendMetadata(maxSeqId, major, mobCells);
         mobFileWriter.close();
         mobStore.commitFile(mobFileWriter.getPath(), path);
       } else {
         // If the mob file is empty, delete it instead of committing.
+        LOG.debug("Aborting writer for {} because there are no MOB cells",
+          mobFileWriter.getPath());
+        // Remove MOB file from reference set
+        mobRefSet.get().remove(mobFileWriter.getPath().getName());
         abortWriter(mobFileWriter);
       }
     }
-    mobStore.updateCellsCountCompactedFromMob(cellsCountCompactedFromMob);
-    mobStore.updateCellsCountCompactedToMob(cellsCountCompactedToMob);
-    mobStore.updateCellsSizeCompactedFromMob(cellsSizeCompactedFromMob);
-    mobStore.updateCellsSizeCompactedToMob(cellsSizeCompactedToMob);
-    progress.complete();
-    return true;
   }
+
+  protected static String createKey(TableName tableName, String encodedName,
+      String columnFamilyName) {
+    return tableName.getNameAsString()+ "_" + encodedName + "_"+ columnFamilyName;
+  }
+
+  @Override
+  protected List<Path> commitWriter(StoreFileWriter writer, FileDetails fd,
+      CompactionRequestImpl request) throws IOException {
+    List<Path> newFiles = Lists.newArrayList(writer.getPath());
+    writer.appendMetadata(fd.maxSeqId, request.isAllFiles(), request.getFiles());
+    // Append MOB references
+    Set<String> refSet = mobRefSet.get();
+    writer.appendMobMetadata(refSet);
+    writer.close();
+    return newFiles;
+  }
+
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreFlusher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreFlusher.java
index 3de7992..70591fb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreFlusher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreFlusher.java
@@ -22,7 +22,9 @@ import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.util.ArrayList;
 import java.util.Date;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Set;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
@@ -69,6 +71,13 @@ public class DefaultMobStoreFlusher extends DefaultStoreFlusher {
   private long mobCellValueSizeThreshold = 0;
   private Path targetPath;
   private HMobStore mobStore;
+  // MOB file reference set
+  static ThreadLocal<Set<String>> mobRefSet = new ThreadLocal<Set<String>>() {
+    @Override
+    protected Set<String> initialValue() {
+      return new HashSet<String>();
+    }
+  };
 
   public DefaultMobStoreFlusher(Configuration conf, HStore store) throws IOException {
     super(conf, store);
@@ -188,6 +197,8 @@ public class DefaultMobStoreFlusher extends DefaultStoreFlusher {
       throughputController.start(flushName);
     }
     IOException ioe = null;
+    // Clear all past MOB references
+    mobRefSet.get().clear();
     try {
       do {
         hasMore = scanner.next(cells, scannerContext);
@@ -204,7 +215,6 @@ public class DefaultMobStoreFlusher extends DefaultStoreFlusher {
               mobFileWriter.append(c);
               mobSize += c.getValueLength();
               mobCount++;
-
               // append the tags to the KeyValue.
               // The key is same, the value is the filename of the mob file
               Cell reference = MobUtils.createMobRefCell(c, fileName,
@@ -244,9 +254,12 @@ public class DefaultMobStoreFlusher extends DefaultStoreFlusher {
       status.setStatus("Flushing mob file " + store + ": closing flushed file");
       mobFileWriter.close();
       mobStore.commitFile(mobFileWriter.getPath(), targetPath);
+      LOG.debug("Flush store file: {}", writer.getPath());
       mobStore.updateMobFlushCount();
       mobStore.updateMobFlushedCellsCount(mobCount);
       mobStore.updateMobFlushedCellsSize(mobSize);
+      // Add mob reference to store file metadata
+      mobRefSet.get().add(mobFileWriter.getPath().getName());
     } else {
       try {
         status.setStatus("Flushing mob file " + store + ": no mob cells, closing flushed file");
@@ -258,4 +271,16 @@ public class DefaultMobStoreFlusher extends DefaultStoreFlusher {
       }
     }
   }
+
+  protected void finalizeWriter(StoreFileWriter writer, long cacheFlushSeqNum,
+      MonitoredTask status) throws IOException {
+    // Write out the log sequence number that corresponds to this output
+    // hfile. Also write current time in metadata as minFlushTime.
+    // The hfile is current up to and including cacheFlushSeqNum.
+    status.setStatus("Flushing " + store + ": appending metadata");
+    writer.appendMetadata(cacheFlushSeqNum, false);
+    writer.appendMobMetadata(mobRefSet.get());
+    status.setStatus("Flushing " + store + ": closing flushed file");
+    writer.close();
+  }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/ExpiredMobFileCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/ExpiredMobFileCleaner.java
index 42e78bb..f82cf1a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/ExpiredMobFileCleaner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/ExpiredMobFileCleaner.java
@@ -57,7 +57,8 @@ public class ExpiredMobFileCleaner extends Configured implements Tool {
    * @param tableName The current table name.
    * @param family The current family.
    */
-  public void cleanExpiredMobFiles(String tableName, ColumnFamilyDescriptor family) throws IOException {
+  public void cleanExpiredMobFiles(String tableName, ColumnFamilyDescriptor family)
+      throws IOException {
     Configuration conf = getConf();
     TableName tn = TableName.valueOf(tableName);
     FileSystem fs = FileSystem.get(conf);
@@ -67,7 +68,7 @@ public class ExpiredMobFileCleaner extends Configured implements Tool {
     copyOfConf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0f);
     CacheConfig cacheConfig = new CacheConfig(copyOfConf);
     MobUtils.cleanExpiredMobFiles(fs, conf, tn, family, cacheConfig,
-        EnvironmentEdgeManager.currentTime());
+      EnvironmentEdgeManager.currentTime());
   }
 
   public static void main(String[] args) throws Exception {
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 4afd75b..fa0944b 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
@@ -43,7 +43,7 @@ public final class MobConstants {
   public static final String MOB_REGION_NAME = ".mob";
   public static final byte[] MOB_REGION_NAME_BYTES = Bytes.toBytes(MOB_REGION_NAME);
 
-  public static final String MOB_CLEANER_PERIOD = "hbase.master.mob.ttl.cleaner.period";
+  public static final String MOB_CLEANER_PERIOD = "hbase.master.mob.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";
@@ -55,33 +55,22 @@ public final class MobConstants {
   public static final long DEFAULT_MOB_CACHE_EVICT_PERIOD = 3600L;
 
   public final static String TEMP_DIR_NAME = ".tmp";
-  public final static String BULKLOAD_DIR_NAME = ".bulkload";
-  public final static byte[] MOB_TABLE_LOCK_SUFFIX = Bytes.toBytes(".mobLock");
-  public final static String EMPTY_STRING = "";
-  /**
-   * If the size of a mob file is less than this value, it's regarded as a small file and needs to
-   * be merged in mob compaction. The default value is 1280MB.
-   */
-  public static final String MOB_COMPACTION_MERGEABLE_THRESHOLD =
-    "hbase.mob.compaction.mergeable.threshold";
-  public static final long DEFAULT_MOB_COMPACTION_MERGEABLE_THRESHOLD = 10 * 128 * 1024 * 1024;
+
   /**
-   * The max number of del files that is allowed in the mob file compaction. In the mob
-   * compaction, when the number of existing del files is larger than this value, they are merged
-   * until number of del files is not larger this value. The default value is 3.
+   * The max number of a MOB table regions that is allowed in a batch of the mob compaction.
+   * By setting this number to a custom value, users can control the overall effect
+   * of a major compaction of a large MOB-enabled table.
    */
-  public static final String MOB_DELFILE_MAX_COUNT = "hbase.mob.delfile.max.count";
-  public static final int DEFAULT_MOB_DELFILE_MAX_COUNT = 3;
+
+  public static final String MOB_MAJOR_COMPACTION_REGION_BATCH_SIZE =
+    "hbase.mob.major.compaction.region.batch.size";
+
   /**
-   * The max number of the mob files that is allowed in a batch of the mob compaction.
-   * The mob compaction merges the small mob files to bigger ones. If the number of the
-   * small files is very large, it could lead to a "too many opened file handlers" in the merge.
-   * And the merge has to be split into batches. This value limits the number of mob files
-   * that are selected in a batch of the mob compaction. The default value is 100.
+   * Default is 0 - means no limit - all regions of a MOB table will be compacted at once
    */
-  public static final String MOB_COMPACTION_BATCH_SIZE =
-    "hbase.mob.compaction.batch.size";
-  public static final int DEFAULT_MOB_COMPACTION_BATCH_SIZE = 100;
+
+  public static final int DEFAULT_MOB_MAJOR_COMPACTION_REGION_BATCH_SIZE = 0;
+
   /**
    * The period that MobCompactionChore runs. The unit is second.
    * The default value is one week.
@@ -91,12 +80,46 @@ public final class MobConstants {
   public static final int DEFAULT_MOB_COMPACTION_CHORE_PERIOD =
     24 * 60 * 60 * 7; // a week
   public static final String MOB_COMPACTOR_CLASS_KEY = "hbase.mob.compactor.class";
+
+  /**
+   * Mob compaction type: "full", "io_optimized"
+   * "full" - run full major compaction (during migration)
+   * "io_optimized" - optimized version for use  case with infrequent updates/deletes
+   */
+  public final static String MOB_COMPACTION_TYPE_KEY = "hbase.mob.compaction.type";
+
+  public final static String DEFAULT_MOB_COMPACTION_TYPE = "full";
+
+  public final static String IO_OPTIMIZED_MOB_COMPACTION_TYPE = "optimized";
+
+  public final static String FULL_MOB_COMPACTION_TYPE = "full";
+
+
+  /**
+   * Maximum size of a MOB compaction selection
+   */
+  public static final String MOB_COMPACTION_MAX_FILE_SIZE_KEY =
+      "hbase.mob.compactions.max.file.size";
+  /**
+   * Default maximum selection size = 1GB
+   */
+  public static final long DEFAULT_MOB_COMPACTION_MAX_FILE_SIZE = 1024 * 1024 * 1024;
+
   /**
-   * The max number of threads used in MobCompactor.
+   * Use this configuration option with caution, only during upgrade procedure
+   * to handle missing MOB cells during compaction.
    */
-  public static final String MOB_COMPACTION_THREADS_MAX =
-    "hbase.mob.compaction.threads.max";
-  public static final int DEFAULT_MOB_COMPACTION_THREADS_MAX = 1;
+  public static final String MOB_UNSAFE_DISCARD_MISS_KEY = "hbase.unsafe.mob.discard.miss";
+
+  public static final boolean DEFAULT_MOB_DISCARD_MISS = false;
+
+  /**
+   * Minimum MOB file age to archive, default (3600000 - 1h)
+   */
+  public static final String MIN_AGE_TO_ARCHIVE_KEY = "hbase.mob.min.age.archive";
+
+  public static final long DEFAULT_MIN_AGE_TO_ARCHIVE = 3600000; // 1h
+
   private MobConstants() {
 
   }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileName.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileName.java
index 3a29274..64cfa4d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileName.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileName.java
@@ -23,8 +23,8 @@ import org.apache.hadoop.hbase.util.MD5Hash;
 
 /**
  * The mob file name.
- * It consists of a md5 of a start key, a date and an uuid.
- * It looks like md5(start) + date + uuid.
+ * It consists of a md5 of a start key, a date, uuid and encoded region name.
+ * It looks like md5(start) + date + uuid+ "_" + encoded region name.
  * <ol>
  * <li>characters 0-31: md5 hex string of a start key. Since the length of the start key is not
  * fixed, have to use the md5 instead which has a fix length.</li>
@@ -45,10 +45,13 @@ public final class MobFileName {
   private final String startKey;
   private final String uuid;
   private final String fileName;
+  // Name of a region this MOB file belongs to
+  private final String regionName;
 
   private static final int STARTKEY_END_INDEX = 32;
   private static final int DATE_END_INDEX = 40;
   private static final int UUID_END_INDEX = 72;
+  public static final String REGION_SEP = "_";
 
   /**
    * @param startKey
@@ -57,12 +60,14 @@ public final class MobFileName {
    *          The string of the latest timestamp of cells in this file, the format is yyyymmdd.
    * @param uuid
    *          The uuid
+   * @param regionName name of a region, where this file was created during flush or compaction.
    */
-  private MobFileName(byte[] startKey, String date, String uuid) {
+  private MobFileName(byte[] startKey, String date, String uuid, String regionName) {
     this.startKey = MD5Hash.getMD5AsHex(startKey, 0, startKey.length);
     this.uuid = uuid;
     this.date = date;
-    this.fileName = this.startKey + this.date + this.uuid;
+    this.regionName = regionName;
+    this.fileName = this.startKey + this.date + this.uuid + REGION_SEP + this.regionName;
   }
 
   /**
@@ -72,12 +77,14 @@ public final class MobFileName {
    *          The string of the latest timestamp of cells in this file, the format is yyyymmdd.
    * @param uuid
    *          The uuid
+   * @param regionName name of a region, where this file was created during flush or compaction.
    */
-  private MobFileName(String startKey, String date, String uuid) {
+  private MobFileName(String startKey, String date, String uuid, String regionName) {
     this.startKey = startKey;
     this.uuid = uuid;
     this.date = date;
-    this.fileName = this.startKey + this.date + this.uuid;
+    this.regionName = regionName;
+    this.fileName = this.startKey + this.date + this.uuid + REGION_SEP + this.regionName;
   }
 
   /**
@@ -88,10 +95,11 @@ public final class MobFileName {
    * @param date
    *          The string of the latest timestamp of cells in this file, the format is yyyymmdd.
    * @param uuid The uuid.
+   * @param regionName name of a region, where this file was created during flush or compaction.
    * @return An instance of a MobFileName.
    */
-  public static MobFileName create(byte[] startKey, String date, String uuid) {
-    return new MobFileName(startKey, date, uuid);
+  public static MobFileName create(byte[] startKey, String date, String uuid, String regionName) {
+    return new MobFileName(startKey, date, uuid, regionName);
   }
 
   /**
@@ -102,10 +110,11 @@ public final class MobFileName {
    * @param date
    *          The string of the latest timestamp of cells in this file, the format is yyyymmdd.
    * @param uuid The uuid.
+   * @param regionName name of a region, where this file was created during flush or compaction.
    * @return An instance of a MobFileName.
    */
-  public static MobFileName create(String startKey, String date, String uuid) {
-    return new MobFileName(startKey, date, uuid);
+  public static MobFileName create(String startKey, String date, String uuid, String regionName) {
+    return new MobFileName(startKey, date, uuid, regionName);
   }
 
   /**
@@ -119,7 +128,8 @@ public final class MobFileName {
     String startKey = fileName.substring(0, STARTKEY_END_INDEX);
     String date = fileName.substring(STARTKEY_END_INDEX, DATE_END_INDEX);
     String uuid = fileName.substring(DATE_END_INDEX, UUID_END_INDEX);
-    return new MobFileName(startKey, date, uuid);
+    String regionName = fileName.substring(UUID_END_INDEX+1);
+    return new MobFileName(startKey, date, uuid, regionName);
   }
 
   /**
@@ -149,6 +159,13 @@ public final class MobFileName {
   }
 
   /**
+   * Gets region name
+   * @return name of a region, where this file was created during flush or compaction.
+   */
+  public String getRegionName() {
+    return regionName;
+  }
+  /**
    * Gets the date string. Its format is yyyymmdd.
    * @return The date string.
    */
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobStoreEngine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobStoreEngine.java
index ee1fe7d..6adb4b5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobStoreEngine.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobStoreEngine.java
@@ -22,6 +22,7 @@ import java.io.IOException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.regionserver.DefaultStoreEngine;
 import org.apache.hadoop.hbase.regionserver.HStore;
+import org.apache.hadoop.hbase.util.ReflectionUtils;
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
@@ -29,7 +30,7 @@ import org.apache.yetus.audience.InterfaceAudience;
  */
 @InterfaceAudience.Private
 public class MobStoreEngine extends DefaultStoreEngine {
-
+  public final static String DEFAULT_MOB_COMPACTOR_CLASS_KEY = "hbase.mob.default.compactor";
   @Override
   protected void createStoreFlusher(Configuration conf, HStore store) throws IOException {
     // When using MOB, we use DefaultMobStoreFlusher always
@@ -43,6 +44,13 @@ public class MobStoreEngine extends DefaultStoreEngine {
    */
   @Override
   protected void createCompactor(Configuration conf, HStore store) throws IOException {
-    compactor = new DefaultMobStoreCompactor(conf, store);
+    String className =
+        conf.get(DEFAULT_MOB_COMPACTOR_CLASS_KEY, DefaultMobStoreCompactor.class.getName());
+    try {
+      compactor = ReflectionUtils.instantiateWithCustomCtor(className,
+        new Class[] { Configuration.class, HStore.class }, new Object[] { conf, store });
+    } catch (Exception e) {
+      throw new IOException("Unable to load configured compactor '" + className + "'", e);
+    }
   }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java
index 1c00e25..f20435d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java
@@ -25,16 +25,14 @@ import java.text.SimpleDateFormat;
 import java.util.ArrayList;
 import java.util.Calendar;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.Date;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Optional;
+import java.util.Set;
 import java.util.UUID;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.RejectedExecutionException;
-import java.util.concurrent.RejectedExecutionHandler;
-import java.util.concurrent.SynchronousQueue;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -48,7 +46,6 @@ import org.apache.hadoop.hbase.TagType;
 import org.apache.hadoop.hbase.TagUtil;
 import org.apache.hadoop.hbase.backup.HFileArchiver;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
-import org.apache.hadoop.hbase.client.MobCompactPartitionPolicy;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.client.Scan;
@@ -60,10 +57,6 @@ import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.io.hfile.HFileContext;
 import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
-import org.apache.hadoop.hbase.master.locking.LockManager;
-import org.apache.hadoop.hbase.mob.compactions.MobCompactor;
-import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionPartitionId;
-import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactor;
 import org.apache.hadoop.hbase.regionserver.BloomType;
 import org.apache.hadoop.hbase.regionserver.HStore;
 import org.apache.hadoop.hbase.regionserver.HStoreFile;
@@ -72,8 +65,6 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ChecksumType;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.hadoop.hbase.util.ReflectionUtils;
-import org.apache.hadoop.hbase.util.Threads;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -85,23 +76,15 @@ import org.slf4j.LoggerFactory;
 public final class MobUtils {
 
   private static final Logger LOG = LoggerFactory.getLogger(MobUtils.class);
-  private final static long WEEKLY_THRESHOLD_MULTIPLIER = 7;
-  private final static long MONTHLY_THRESHOLD_MULTIPLIER = 4 * WEEKLY_THRESHOLD_MULTIPLIER;
+  public static final String SEP = "_";
 
   private static final ThreadLocal<SimpleDateFormat> LOCAL_FORMAT =
       new ThreadLocal<SimpleDateFormat>() {
-    @Override
-    protected SimpleDateFormat initialValue() {
-      return new SimpleDateFormat("yyyyMMdd");
-    }
-  };
-
-  private static final byte[] REF_DELETE_MARKER_TAG_BYTES;
-  static {
-    List<Tag> tags = new ArrayList<>();
-    tags.add(MobConstants.MOB_REF_TAG);
-    REF_DELETE_MARKER_TAG_BYTES = TagUtil.fromList(tags);
-  }
+        @Override
+        protected SimpleDateFormat initialValue() {
+          return new SimpleDateFormat("yyyyMMdd");
+        }
+      };
 
   /**
    * Private constructor to keep this class from being instantiated.
@@ -128,45 +111,6 @@ public final class MobUtils {
   }
 
   /**
-   * Get the first day of the input date's month
-   * @param calendar Calendar object
-   * @param date The date to find out its first day of that month
-   * @return The first day in the month
-   */
-  public static Date getFirstDayOfMonth(final Calendar calendar, final Date date) {
-
-    calendar.setTime(date);
-    calendar.set(Calendar.HOUR_OF_DAY, 0);
-    calendar.set(Calendar.MINUTE, 0);
-    calendar.set(Calendar.SECOND, 0);
-    calendar.set(Calendar.MILLISECOND, 0);
-    calendar.set(Calendar.DAY_OF_MONTH, 1);
-
-    Date firstDayInMonth = calendar.getTime();
-    return firstDayInMonth;
-  }
-
-  /**
-   * Get the first day of the input date's week
-   * @param calendar Calendar object
-   * @param date The date to find out its first day of that week
-   * @return The first day in the week
-   */
-  public static Date getFirstDayOfWeek(final Calendar calendar, final Date date) {
-
-    calendar.setTime(date);
-    calendar.set(Calendar.HOUR_OF_DAY, 0);
-    calendar.set(Calendar.MINUTE, 0);
-    calendar.set(Calendar.SECOND, 0);
-    calendar.set(Calendar.MILLISECOND, 0);
-    calendar.setFirstDayOfWeek(Calendar.MONDAY);
-    calendar.set(Calendar.DAY_OF_WEEK, Calendar.MONDAY);
-
-    Date firstDayInWeek = calendar.getTime();
-    return firstDayInWeek;
-  }
-
-  /**
    * Whether the current cell is a mob reference cell.
    * @param cell The current cell.
    * @return True if the cell has a mob reference tag, false if it doesn't.
@@ -213,11 +157,10 @@ public final class MobUtils {
   }
 
   /**
-   * Indicates whether it's a raw scan.
-   * The information is set in the attribute "hbase.mob.scan.raw" of scan.
-   * For a mob cell, in a normal scan the scanners retrieves the mob cell from the mob file.
-   * In a raw scan, the scanner directly returns cell in HBase without retrieve the one in
-   * the mob file.
+   * Indicates whether it's a raw scan. The information is set in the attribute "hbase.mob.scan.raw"
+   * of scan. For a mob cell, in a normal scan the scanners retrieves the mob cell from the mob
+   * file. In a raw scan, the scanner directly returns cell in HBase without retrieve the one in the
+   * mob file.
    * @param scan The current scan.
    * @return True if it's a raw scan.
    */
@@ -231,9 +174,9 @@ public final class MobUtils {
   }
 
   /**
-   * Indicates whether it's a reference only scan.
-   * The information is set in the attribute "hbase.mob.scan.ref.only" of scan.
-   * If it's a ref only scan, only the cells with ref tag are returned.
+   * Indicates whether it's a reference only scan. The information is set in the attribute
+   * "hbase.mob.scan.ref.only" of scan. If it's a ref only scan, only the cells with ref tag are
+   * returned.
    * @param scan The current scan.
    * @return True if it's a ref only scan.
    */
@@ -247,8 +190,8 @@ public final class MobUtils {
   }
 
   /**
-   * Indicates whether the scan contains the information of caching blocks.
-   * The information is set in the attribute "hbase.mob.cache.blocks" of scan.
+   * Indicates whether the scan contains the information of caching blocks. The information is set
+   * in the attribute "hbase.mob.cache.blocks" of scan.
    * @param scan The current scan.
    * @return True when the Scan attribute specifies to cache the MOB blocks.
    */
@@ -263,22 +206,17 @@ public final class MobUtils {
 
   /**
    * Sets the attribute of caching blocks in the scan.
-   *
-   * @param scan
-   *          The current scan.
-   * @param cacheBlocks
-   *          True, set the attribute of caching blocks into the scan, the scanner with this scan
-   *          caches blocks.
-   *          False, the scanner doesn't cache blocks for this scan.
+   * @param scan The current scan.
+   * @param cacheBlocks True, set the attribute of caching blocks into the scan, the scanner with
+   *          this scan caches blocks. False, the scanner doesn't cache blocks for this scan.
    */
   public static void setCacheMobBlocks(Scan scan, boolean cacheBlocks) {
     scan.setAttribute(MobConstants.MOB_CACHE_BLOCKS, Bytes.toBytes(cacheBlocks));
   }
 
   /**
-   * Cleans the expired mob files.
-   * Cleans the files whose creation date is older than (current - columnFamily.ttl), and
-   * the minVersions of that column family is 0.
+   * Cleans the expired mob files. Cleans the files whose creation date is older than (current -
+   * columnFamily.ttl), and the minVersions of that column family is 0.
    * @param fs The current file system.
    * @param conf The current configuration.
    * @param tableName The current table name.
@@ -330,11 +268,11 @@ public final class MobUtils {
         Date fileDate = parseDate(MobFileName.getDateFromName(fileName));
 
         if (LOG.isDebugEnabled()) {
-          LOG.debug("Checking file " + fileName);
+          LOG.debug("Checking file {}", fileName);
         }
         if (fileDate.getTime() < expireDate.getTime()) {
           if (LOG.isDebugEnabled()) {
-            LOG.debug(fileName + " is an expired file");
+            LOG.debug("{} is an expired file", fileName);
           }
           filesToClean
               .add(new HStoreFile(fs, file.getPath(), conf, cacheConfig, BloomType.NONE, true));
@@ -345,19 +283,17 @@ public final class MobUtils {
     }
     if (!filesToClean.isEmpty()) {
       try {
-        removeMobFiles(conf, fs, tableName, mobTableDir, columnDescriptor.getName(),
-            filesToClean);
+        removeMobFiles(conf, fs, tableName, mobTableDir, columnDescriptor.getName(), filesToClean);
         deletedFileCount = filesToClean.size();
       } catch (IOException e) {
         LOG.error("Failed to delete the mob files " + filesToClean, e);
       }
     }
-    LOG.info(deletedFileCount + " expired mob files are deleted");
+    LOG.info("{} expired mob files are deleted", deletedFileCount);
   }
 
   /**
-   * Gets the root dir of the mob files.
-   * It's {HBASE_DIR}/mobdir.
+   * Gets the root dir of the mob files. It's {HBASE_DIR}/mobdir.
    * @param conf The current configuration.
    * @return the root dir of the mob file.
    */
@@ -367,8 +303,7 @@ public final class MobUtils {
   }
 
   /**
-   * Gets the root dir of the mob files under the qualified HBase root dir.
-   * It's {rootDir}/mobdir.
+   * Gets the root dir of the mob files under the qualified HBase root dir. It's {rootDir}/mobdir.
    * @param rootDir The qualified path of HBase root directory.
    * @return The root dir of the mob file.
    */
@@ -389,8 +324,8 @@ public final class MobUtils {
   }
 
   /**
-   * Gets the table dir of the mob files under the qualified HBase root dir.
-   * It's {rootDir}/mobdir/data/${namespace}/${tableName}
+   * Gets the table dir of the mob files under the qualified HBase root dir. It's
+   * {rootDir}/mobdir/data/${namespace}/${tableName}
    * @param rootDir The qualified path of HBase root directory.
    * @param tableName The name of table.
    * @return The table dir of the mob file.
@@ -400,8 +335,8 @@ public final class MobUtils {
   }
 
   /**
-   * Gets the region dir of the mob files.
-   * It's {HBASE_DIR}/mobdir/data/{namespace}/{tableName}/{regionEncodedName}.
+   * Gets the region dir of the mob files. It's
+   * {HBASE_DIR}/mobdir/data/{namespace}/{tableName}/{regionEncodedName}.
    * @param conf The current configuration.
    * @param tableName The current table name.
    * @return The region dir of the mob files.
@@ -411,8 +346,8 @@ public final class MobUtils {
   }
 
   /**
-   * Gets the region dir of the mob files under the specified root dir.
-   * It's {rootDir}/mobdir/data/{namespace}/{tableName}/{regionEncodedName}.
+   * Gets the region dir of the mob files under the specified root dir. It's
+   * {rootDir}/mobdir/data/{namespace}/{tableName}/{regionEncodedName}.
    * @param rootDir The qualified path of HBase root directory.
    * @param tableName The current table name.
    * @return The region dir of the mob files.
@@ -424,8 +359,8 @@ public final class MobUtils {
   }
 
   /**
-   * Gets the family dir of the mob files.
-   * It's {HBASE_DIR}/mobdir/{namespace}/{tableName}/{regionEncodedName}/{columnFamilyName}.
+   * Gets the family dir of the mob files. It's
+   * {HBASE_DIR}/mobdir/{namespace}/{tableName}/{regionEncodedName}/{columnFamilyName}.
    * @param conf The current configuration.
    * @param tableName The current table name.
    * @param familyName The current family name.
@@ -436,8 +371,8 @@ public final class MobUtils {
   }
 
   /**
-   * Gets the family dir of the mob files.
-   * It's {HBASE_DIR}/mobdir/{namespace}/{tableName}/{regionEncodedName}/{columnFamilyName}.
+   * Gets the family dir of the mob files. It's
+   * {HBASE_DIR}/mobdir/{namespace}/{tableName}/{regionEncodedName}/{columnFamilyName}.
    * @param regionPath The path of mob region which is a dummy one.
    * @param familyName The current family name.
    * @return The family dir of the mob files.
@@ -447,19 +382,14 @@ public final class MobUtils {
   }
 
   /**
-   * Gets the RegionInfo of the mob files.
-   * This is a dummy region. The mob files are not saved in a region in HBase.
-   * This is only used in mob snapshot. It's internally used only.
+   * Gets the RegionInfo of the mob files. This is a dummy region. The mob files are not saved in a
+   * region in HBase. This is only used in mob snapshot. It's internally used only.
    * @param tableName
    * @return A dummy mob region info.
    */
   public static RegionInfo getMobRegionInfo(TableName tableName) {
-    return RegionInfoBuilder.newBuilder(tableName)
-        .setStartKey(MobConstants.MOB_REGION_NAME_BYTES)
-        .setEndKey(HConstants.EMPTY_END_ROW)
-        .setSplit(false)
-        .setRegionId(0)
-        .build();
+    return RegionInfoBuilder.newBuilder(tableName).setStartKey(MobConstants.MOB_REGION_NAME_BYTES)
+        .setEndKey(HConstants.EMPTY_END_ROW).setSplit(false).setRegionId(0).build();
   }
 
   /**
@@ -468,8 +398,9 @@ public final class MobUtils {
    * @return If true, the current RegionInfo is a mob one.
    */
   public static boolean isMobRegionInfo(RegionInfo regionInfo) {
-    return regionInfo == null ? false : getMobRegionInfo(regionInfo.getTable()).getEncodedName()
-        .equals(regionInfo.getEncodedName());
+    return regionInfo == null ? false
+        : getMobRegionInfo(regionInfo.getTable()).getEncodedName()
+            .equals(regionInfo.getEncodedName());
   }
 
   /**
@@ -483,16 +414,6 @@ public final class MobUtils {
   }
 
   /**
-   * Gets the working directory of the mob compaction.
-   * @param root The root directory of the mob compaction.
-   * @param jobName The current job name.
-   * @return The directory of the mob compaction for the current job.
-   */
-  public static Path getCompactionWorkingPath(Path root, String jobName) {
-    return new Path(root, jobName);
-  }
-
-  /**
    * Archives the mob files.
    * @param conf The current configuration.
    * @param fs The current file system.
@@ -504,16 +425,18 @@ public final class MobUtils {
   public static void removeMobFiles(Configuration conf, FileSystem fs, TableName tableName,
       Path tableDir, byte[] family, Collection<HStoreFile> storeFiles) throws IOException {
     HFileArchiver.archiveStoreFiles(conf, fs, getMobRegionInfo(tableName), tableDir, family,
-        storeFiles);
+      storeFiles);
   }
 
+
+
   /**
-   * Creates a mob reference KeyValue.
-   * The value of the mob reference KeyValue is mobCellValueSize + mobFileName.
+   * Creates a mob reference KeyValue. The value of the mob reference KeyValue is mobCellValueSize +
+   * mobFileName.
    * @param cell The original Cell.
    * @param fileName The mob file name where the mob reference KeyValue is written.
-   * @param tableNameTag The tag of the current table name. It's very important in
-   *                        cloning the snapshot.
+   * @param tableNameTag The tag of the current table name. It's very important in cloning the
+   *          snapshot.
    * @return The mob reference KeyValue.
    */
   public static Cell createMobRefCell(Cell cell, byte[] fileName, Tag tableNameTag) {
@@ -554,90 +477,15 @@ public final class MobUtils {
   public static StoreFileWriter createWriter(Configuration conf, FileSystem fs,
       ColumnFamilyDescriptor family, String date, Path basePath, long maxKeyCount,
       Compression.Algorithm compression, String startKey, CacheConfig cacheConfig,
-      Encryption.Context cryptoContext, boolean isCompaction)
+      Encryption.Context cryptoContext, boolean isCompaction, String regionName)
       throws IOException {
     MobFileName mobFileName = MobFileName.create(startKey, date,
-        UUID.randomUUID().toString().replaceAll("-", ""));
-    return createWriter(conf, fs, family, mobFileName, basePath, maxKeyCount, compression,
-        cacheConfig, cryptoContext, isCompaction);
-  }
-
-  /**
-   * Creates a writer for the ref file in temp directory.
-   * @param conf The current configuration.
-   * @param fs The current file system.
-   * @param family The descriptor of the current column family.
-   * @param basePath The basic path for a temp directory.
-   * @param maxKeyCount The key count.
-   * @param cacheConfig The current cache config.
-   * @param cryptoContext The encryption context.
-   * @param isCompaction If the writer is used in compaction.
-   * @return The writer for the mob file.
-   */
-  public static StoreFileWriter createRefFileWriter(Configuration conf, FileSystem fs,
-    ColumnFamilyDescriptor family, Path basePath, long maxKeyCount, CacheConfig cacheConfig,
-    Encryption.Context cryptoContext, boolean isCompaction)
-    throws IOException {
-    return createWriter(conf, fs, family,
-      new Path(basePath, UUID.randomUUID().toString().replaceAll("-", "")), maxKeyCount,
-      family.getCompactionCompressionType(), cacheConfig, cryptoContext,
-      HStore.getChecksumType(conf), HStore.getBytesPerChecksum(conf), family.getBlocksize(),
-      family.getBloomFilterType(), isCompaction);
-  }
-
-  /**
-   * Creates a writer for the mob file in temp directory.
-   * @param conf The current configuration.
-   * @param fs The current file system.
-   * @param family The descriptor of the current column family.
-   * @param date The date string, its format is yyyymmmdd.
-   * @param basePath The basic path for a temp directory.
-   * @param maxKeyCount The key count.
-   * @param compression The compression algorithm.
-   * @param startKey The start key.
-   * @param cacheConfig The current cache config.
-   * @param cryptoContext The encryption context.
-   * @param isCompaction If the writer is used in compaction.
-   * @return The writer for the mob file.
-   */
-  public static StoreFileWriter createWriter(Configuration conf, FileSystem fs,
-      ColumnFamilyDescriptor family, String date, Path basePath, long maxKeyCount,
-      Compression.Algorithm compression, byte[] startKey, CacheConfig cacheConfig,
-      Encryption.Context cryptoContext, boolean isCompaction)
-      throws IOException {
-    MobFileName mobFileName = MobFileName.create(startKey, date,
-        UUID.randomUUID().toString().replaceAll("-", ""));
+      UUID.randomUUID().toString().replaceAll("-", ""), regionName);
     return createWriter(conf, fs, family, mobFileName, basePath, maxKeyCount, compression,
       cacheConfig, cryptoContext, isCompaction);
   }
 
   /**
-   * Creates a writer for the del file in temp directory.
-   * @param conf The current configuration.
-   * @param fs The current file system.
-   * @param family The descriptor of the current column family.
-   * @param date The date string, its format is yyyymmmdd.
-   * @param basePath The basic path for a temp directory.
-   * @param maxKeyCount The key count.
-   * @param compression The compression algorithm.
-   * @param startKey The start key.
-   * @param cacheConfig The current cache config.
-   * @param cryptoContext The encryption context.
-   * @return The writer for the del file.
-   */
-  public static StoreFileWriter createDelFileWriter(Configuration conf, FileSystem fs,
-      ColumnFamilyDescriptor family, String date, Path basePath, long maxKeyCount,
-      Compression.Algorithm compression, byte[] startKey, CacheConfig cacheConfig,
-      Encryption.Context cryptoContext)
-      throws IOException {
-    String suffix = UUID
-      .randomUUID().toString().replaceAll("-", "") + "_del";
-    MobFileName mobFileName = MobFileName.create(startKey, date, suffix);
-    return createWriter(conf, fs, family, mobFileName, basePath, maxKeyCount, compression,
-      cacheConfig, cryptoContext, true);
-  }
-
-  /**
    * Creates a writer for the mob file in temp directory.
    * @param conf The current configuration.
    * @param fs The current file system.
@@ -654,12 +502,10 @@ public final class MobUtils {
   public static StoreFileWriter createWriter(Configuration conf, FileSystem fs,
       ColumnFamilyDescriptor family, MobFileName mobFileName, Path basePath, long maxKeyCount,
       Compression.Algorithm compression, CacheConfig cacheConfig, Encryption.Context cryptoContext,
-      boolean isCompaction)
-      throws IOException {
-    return createWriter(conf, fs, family,
-      new Path(basePath, mobFileName.getFileName()), maxKeyCount, compression, cacheConfig,
-      cryptoContext, HStore.getChecksumType(conf), HStore.getBytesPerChecksum(conf),
-      family.getBlocksize(), BloomType.NONE, isCompaction);
+      boolean isCompaction) throws IOException {
+    return createWriter(conf, fs, family, new Path(basePath, mobFileName.getFileName()),
+      maxKeyCount, compression, cacheConfig, cryptoContext, HStore.getChecksumType(conf),
+      HStore.getBytesPerChecksum(conf), family.getBlocksize(), BloomType.NONE, isCompaction);
   }
 
   /**
@@ -680,10 +526,9 @@ public final class MobUtils {
    * @return The writer for the mob file.
    */
   public static StoreFileWriter createWriter(Configuration conf, FileSystem fs,
-      ColumnFamilyDescriptor family, Path path, long maxKeyCount,
-      Compression.Algorithm compression, CacheConfig cacheConfig, Encryption.Context cryptoContext,
-      ChecksumType checksumType, int bytesPerChecksum, int blocksize, BloomType bloomType,
-      boolean isCompaction)
+      ColumnFamilyDescriptor family, Path path, long maxKeyCount, Compression.Algorithm compression,
+      CacheConfig cacheConfig, Encryption.Context cryptoContext, ChecksumType checksumType,
+      int bytesPerChecksum, int blocksize, BloomType bloomType, boolean isCompaction)
       throws IOException {
     if (compression == null) {
       compression = HFile.DEFAULT_COMPRESSION_ALGORITHM;
@@ -696,13 +541,10 @@ public final class MobUtils {
       writerCacheConf = cacheConfig;
     }
     HFileContext hFileContext = new HFileContextBuilder().withCompression(compression)
-        .withIncludesMvcc(true).withIncludesTags(true)
-        .withCompressTags(family.isCompressTags())
-        .withChecksumType(checksumType)
-        .withBytesPerCheckSum(bytesPerChecksum)
-        .withBlockSize(blocksize)
-        .withHBaseCheckSum(true).withDataBlockEncoding(family.getDataBlockEncoding())
-        .withEncryptionContext(cryptoContext)
+        .withIncludesMvcc(true).withIncludesTags(true).withCompressTags(family.isCompressTags())
+        .withChecksumType(checksumType).withBytesPerCheckSum(bytesPerChecksum)
+        .withBlockSize(blocksize).withHBaseCheckSum(true)
+        .withDataBlockEncoding(family.getDataBlockEncoding()).withEncryptionContext(cryptoContext)
         .withCreateTime(EnvironmentEdgeManager.currentTime()).build();
 
     StoreFileWriter w = new StoreFileWriter.Builder(conf, writerCacheConf, fs)
@@ -712,62 +554,9 @@ public final class MobUtils {
   }
 
   /**
-   * Commits the mob file.
-   * @param conf The current configuration.
-   * @param fs The current file system.
-   * @param sourceFile The path where the mob file is saved.
-   * @param targetPath The directory path where the source file is renamed to.
-   * @param cacheConfig The current cache config.
-   * @return The target file path the source file is renamed to.
-   */
-  public static Path commitFile(Configuration conf, FileSystem fs, final Path sourceFile,
-      Path targetPath, CacheConfig cacheConfig) throws IOException {
-    if (sourceFile == null) {
-      return null;
-    }
-    Path dstPath = new Path(targetPath, sourceFile.getName());
-    validateMobFile(conf, fs, sourceFile, cacheConfig, true);
-    String msg = "Renaming flushed file from " + sourceFile + " to " + dstPath;
-    LOG.info(msg);
-    Path parent = dstPath.getParent();
-    if (!fs.exists(parent)) {
-      fs.mkdirs(parent);
-    }
-    if (!fs.rename(sourceFile, dstPath)) {
-      throw new IOException("Failed rename of " + sourceFile + " to " + dstPath);
-    }
-    return dstPath;
-  }
-
-  /**
-   * Validates a mob file by opening and closing it.
-   * @param conf The current configuration.
-   * @param fs The current file system.
-   * @param path The path where the mob file is saved.
-   * @param cacheConfig The current cache config.
-   */
-  private static void validateMobFile(Configuration conf, FileSystem fs, Path path,
-      CacheConfig cacheConfig, boolean primaryReplica) throws IOException {
-    HStoreFile storeFile = null;
-    try {
-      storeFile = new HStoreFile(fs, path, conf, cacheConfig, BloomType.NONE, primaryReplica);
-      storeFile.initReader();
-    } catch (IOException e) {
-      LOG.error("Failed to open mob file[" + path + "], keep it in temp directory.", e);
-      throw e;
-    } finally {
-      if (storeFile != null) {
-        storeFile.closeStoreFile(false);
-      }
-    }
-  }
-
-  /**
-   * Indicates whether the current mob ref cell has a valid value.
-   * A mob ref cell has a mob reference tag.
-   * The value of a mob ref cell consists of two parts, real mob value length and mob file name.
-   * The real mob value length takes 4 bytes.
-   * The remaining part is the mob file name.
+   * Indicates whether the current mob ref cell has a valid value. A mob ref cell has a mob
+   * reference tag. The value of a mob ref cell consists of two parts, real mob value length and mob
+   * file name. The real mob value length takes 4 bytes. The remaining part is the mob file name.
    * @param cell The mob ref cell.
    * @return True if the cell has a valid value.
    */
@@ -776,11 +565,9 @@ public final class MobUtils {
   }
 
   /**
-   * Gets the mob value length from the mob ref cell.
-   * A mob ref cell has a mob reference tag.
-   * The value of a mob ref cell consists of two parts, real mob value length and mob file name.
-   * The real mob value length takes 4 bytes.
-   * The remaining part is the mob file name.
+   * Gets the mob value length from the mob ref cell. A mob ref cell has a mob reference tag. The
+   * value of a mob ref cell consists of two parts, real mob value length and mob file name. The
+   * real mob value length takes 4 bytes. The remaining part is the mob file name.
    * @param cell The mob ref cell.
    * @return The real mob value length.
    */
@@ -789,98 +576,15 @@ public final class MobUtils {
   }
 
   /**
-   * Gets the mob file name from the mob ref cell.
-   * A mob ref cell has a mob reference tag.
-   * The value of a mob ref cell consists of two parts, real mob value length and mob file name.
-   * The real mob value length takes 4 bytes.
-   * The remaining part is the mob file name.
+   * Gets the mob file name from the mob ref cell. A mob ref cell has a mob reference tag. The value
+   * of a mob ref cell consists of two parts, real mob value length and mob file name. The real mob
+   * value length takes 4 bytes. The remaining part is the mob file name.
    * @param cell The mob ref cell.
    * @return The mob file name.
    */
   public static String getMobFileName(Cell cell) {
     return Bytes.toString(cell.getValueArray(), cell.getValueOffset() + Bytes.SIZEOF_INT,
-        cell.getValueLength() - Bytes.SIZEOF_INT);
-  }
-
-  /**
-   * Gets the table name used in the table lock.
-   * The table lock name is a dummy one, it's not a table name. It's tableName + ".mobLock".
-   * @param tn The table name.
-   * @return The table name used in table lock.
-   */
-  public static TableName getTableLockName(TableName tn) {
-    byte[] tableName = tn.getName();
-    return TableName.valueOf(Bytes.add(tableName, MobConstants.MOB_TABLE_LOCK_SUFFIX));
-  }
-
-  /**
-   * Performs the mob compaction.
-   * @param conf the Configuration
-   * @param fs the file system
-   * @param tableName the table the compact
-   * @param hcd the column descriptor
-   * @param pool the thread pool
-   * @param allFiles Whether add all mob files into the compaction.
-   */
-  public static void doMobCompaction(Configuration conf, FileSystem fs, TableName tableName,
-        ColumnFamilyDescriptor hcd, ExecutorService pool, boolean allFiles,
-        LockManager.MasterLock lock)
-      throws IOException {
-    String className = conf.get(MobConstants.MOB_COMPACTOR_CLASS_KEY,
-        PartitionedMobCompactor.class.getName());
-    // instantiate the mob compactor.
-    MobCompactor compactor = null;
-    try {
-      compactor = ReflectionUtils.instantiateWithCustomCtor(className, new Class[] {
-        Configuration.class, FileSystem.class, TableName.class, ColumnFamilyDescriptor.class,
-        ExecutorService.class }, new Object[] { conf, fs, tableName, hcd, pool });
-    } catch (Exception e) {
-      throw new IOException("Unable to load configured mob file compactor '" + className + "'", e);
-    }
-    // compact only for mob-enabled column.
-    // obtain a write table lock before performing compaction to avoid race condition
-    // with major compaction in mob-enabled column.
-    try {
-      lock.acquire();
-      LOG.info("start MOB compaction of files for table='{}', column='{}', allFiles={}, " +
-          "compactor='{}'", tableName, hcd.getNameAsString(), allFiles, compactor.getClass());
-      compactor.compact(allFiles);
-    } catch (Exception e) {
-      LOG.error("Failed to compact the mob files for the column " + hcd.getNameAsString()
-          + " in the table " + tableName.getNameAsString(), e);
-    } finally {
-      LOG.info("end MOB compaction of files for table='{}', column='{}', allFiles={}, " +
-          "compactor='{}'", tableName, hcd.getNameAsString(), allFiles, compactor.getClass());
-      lock.release();
-    }
-  }
-
-  /**
-   * Creates a thread pool.
-   * @param conf the Configuration
-   * @return A thread pool.
-   */
-  public static ExecutorService createMobCompactorThreadPool(Configuration conf) {
-    int maxThreads = conf.getInt(MobConstants.MOB_COMPACTION_THREADS_MAX,
-        MobConstants.DEFAULT_MOB_COMPACTION_THREADS_MAX);
-    if (maxThreads == 0) {
-      maxThreads = 1;
-    }
-    final SynchronousQueue<Runnable> queue = new SynchronousQueue<>();
-    ThreadPoolExecutor pool = new ThreadPoolExecutor(1, maxThreads, 60, TimeUnit.SECONDS, queue,
-      Threads.newDaemonThreadFactory("MobCompactor"), new RejectedExecutionHandler() {
-        @Override
-        public void rejectedExecution(Runnable r, ThreadPoolExecutor executor) {
-          try {
-            // waiting for a thread to pick up instead of throwing exceptions.
-            queue.put(r);
-          } catch (InterruptedException e) {
-            throw new RejectedExecutionException(e);
-          }
-        }
-      });
-    ((ThreadPoolExecutor) pool).allowCoreThreadTimeOut(true);
-    return pool;
+      cell.getValueLength() - Bytes.SIZEOF_INT);
   }
 
   /**
@@ -899,6 +603,23 @@ public final class MobUtils {
   }
 
   /**
+   * Get list of Mob column families (if any exists)
+   * @param htd table descriptor
+   * @return list of Mob column families
+   */
+  public static List<ColumnFamilyDescriptor> getMobColumnFamilies(TableDescriptor htd) {
+
+    List<ColumnFamilyDescriptor> fams = new ArrayList<ColumnFamilyDescriptor>();
+    ColumnFamilyDescriptor[] hcds = htd.getColumnFamilies();
+    for (ColumnFamilyDescriptor hcd : hcds) {
+      if (hcd.isMobEnabled()) {
+        fams.add(hcd);
+      }
+    }
+    return fams;
+  }
+
+  /**
    * Indicates whether return null value when the mob file is missing or corrupt.
    * The information is set in the attribute "empty.value.on.mobcell.miss" of scan.
    * @param scan The current scan.
@@ -906,7 +627,7 @@ public final class MobUtils {
    */
   public static boolean isReadEmptyValueOnMobCellMiss(Scan scan) {
     byte[] readEmptyValueOnMobCellMiss =
-      scan.getAttribute(MobConstants.EMPTY_VALUE_ON_MOBCELL_MISS);
+        scan.getAttribute(MobConstants.EMPTY_VALUE_ON_MOBCELL_MISS);
     try {
       return readEmptyValueOnMobCellMiss != null && Bytes.toBoolean(readEmptyValueOnMobCellMiss);
     } catch (IllegalArgumentException e) {
@@ -915,15 +636,6 @@ public final class MobUtils {
   }
 
   /**
-   * Creates a mob ref delete marker.
-   * @param cell The current delete marker.
-   * @return A delete marker with the ref tag.
-   */
-  public static Cell createMobRefDeleteMarker(Cell cell) {
-    return PrivateCellUtil.createCell(cell, TagUtil.concatTags(REF_DELETE_MARKER_TAG_BYTES, cell));
-  }
-
-  /**
    * Checks if the mob file is expired.
    * @param column The descriptor of the current column family.
    * @param current The current time.
@@ -955,85 +667,41 @@ public final class MobUtils {
   }
 
   /**
-   * fill out partition id based on compaction policy and date, threshold...
-   * @param id Partition id to be filled out
-   * @param firstDayOfCurrentMonth The first day in the current month
-   * @param firstDayOfCurrentWeek The first day in the current week
-   * @param dateStr Date string from the mob file
-   * @param policy Mob compaction policy
-   * @param calendar Calendar object
-   * @param threshold Mob compaciton threshold configured
-   * @return true if the file needs to be excluded from compaction
+   * Gets encoded region name from a MOB file name
+   * @param mobFileName MOB file name
+   * @return encoded region name or null
    */
-  public static boolean fillPartitionId(final CompactionPartitionId id,
-      final Date firstDayOfCurrentMonth, final Date firstDayOfCurrentWeek, final String dateStr,
-      final MobCompactPartitionPolicy policy, final Calendar calendar, final long threshold) {
-
-    boolean skipCompcation = false;
-    id.setThreshold(threshold);
-    if (threshold <= 0) {
-      id.setDate(dateStr);
-      return skipCompcation;
+  public static String getEncodedRegionName(String mobFileName) {
+    int index = mobFileName.lastIndexOf(MobFileName.REGION_SEP);
+    if (index < 0) {
+      return null;
     }
+    return mobFileName.substring(index+1);
+  }
 
-    long finalThreshold;
-    Date date;
-    try {
-      date = MobUtils.parseDate(dateStr);
-    } catch (ParseException e)  {
-      LOG.warn("Failed to parse date " + dateStr, e);
-      id.setDate(dateStr);
-      return true;
-    }
+  /**
+   * Get list of referenced MOB files from a given collection
+   * of store files
+   * @param storeFiles store files
+   * @param mobDir MOB file directory
+   * @return list of MOB file paths
+   */
 
-    /* The algorithm works as follows:
-     *    For monthly policy:
-     *       1). If the file's date is in past months, apply 4 * 7 * threshold
-     *       2). If the file's date is in past weeks, apply 7 * threshold
-     *       3). If the file's date is in current week, exclude it from the compaction
-     *    For weekly policy:
-     *       1). If the file's date is in past weeks, apply 7 * threshold
-     *       2). If the file's date in currently, apply threshold
-     *    For daily policy:
-     *       1). apply threshold
-     */
-    if (policy == MobCompactPartitionPolicy.MONTHLY) {
-      if (date.before(firstDayOfCurrentMonth)) {
-        // Check overflow
-        if (threshold < (Long.MAX_VALUE / MONTHLY_THRESHOLD_MULTIPLIER)) {
-          finalThreshold = MONTHLY_THRESHOLD_MULTIPLIER * threshold;
-        } else {
-          finalThreshold = Long.MAX_VALUE;
-        }
-        id.setThreshold(finalThreshold);
+  public static List<Path> getReferencedMobFiles(Collection<HStoreFile> storeFiles, Path mobDir) {
 
-        // set to the date for the first day of that month
-        id.setDate(MobUtils.formatDate(MobUtils.getFirstDayOfMonth(calendar, date)));
-        return skipCompcation;
+    Set<String> mobSet = new HashSet<String>();
+    for (HStoreFile sf : storeFiles) {
+      byte[] value = sf.getMetadataValue(HStoreFile.MOB_FILE_REFS);
+      if (value != null && value.length > 1) {
+        String s = Bytes.toString(value);
+        String[] all = s.split(",");
+        Collections.addAll(mobSet, all);
       }
     }
-
-    if ((policy == MobCompactPartitionPolicy.MONTHLY) ||
-        (policy == MobCompactPartitionPolicy.WEEKLY)) {
-      // Check if it needs to apply weekly multiplier
-      if (date.before(firstDayOfCurrentWeek)) {
-        // Check overflow
-        if (threshold < (Long.MAX_VALUE / WEEKLY_THRESHOLD_MULTIPLIER)) {
-          finalThreshold = WEEKLY_THRESHOLD_MULTIPLIER * threshold;
-        } else {
-          finalThreshold = Long.MAX_VALUE;
-        }
-        id.setThreshold(finalThreshold);
-
-        id.setDate(MobUtils.formatDate(MobUtils.getFirstDayOfWeek(calendar, date)));
-        return skipCompcation;
-      } else if (policy == MobCompactPartitionPolicy.MONTHLY) {
-        skipCompcation = true;
-      }
+    List<Path> retList = new ArrayList<Path>();
+    for (String name : mobSet) {
+      retList.add(new Path(mobDir, name));
     }
-
-    // Rest is daily
-    id.setDate(dateStr);
-    return skipCompcation;
+    return retList;
   }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/MobCompactionRequest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/MobCompactionRequest.java
deleted file mode 100644
index 5025ed2..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/MobCompactionRequest.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.mob.compactions;
-
-import org.apache.yetus.audience.InterfaceAudience;
-
-/**
- * The compaction request for mob files.
- */
-@InterfaceAudience.Private
-public abstract class MobCompactionRequest {
-
-  protected long selectionTime;
-  protected CompactionType type = CompactionType.PART_FILES;
-
-  public void setCompactionType(CompactionType type) {
-    this.type = type;
-  }
-
-  /**
-   * Gets the selection time.
-   * @return The selection time.
-   */
-  public long getSelectionTime() {
-    return this.selectionTime;
-  }
-
-  /**
-   * Gets the compaction type.
-   * @return The compaction type.
-   */
-  public CompactionType getCompactionType() {
-    return type;
-  }
-
-  protected enum CompactionType {
-
-    /**
-     * Part of mob files are selected.
-     */
-    PART_FILES,
-
-    /**
-     * All of mob files are selected.
-     */
-    ALL_FILES
-  }
-}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/MobCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/MobCompactor.java
deleted file mode 100644
index c5d93ea..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/MobCompactor.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.mob.compactions;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.List;
-import java.util.concurrent.ExecutorService;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
-import org.apache.hadoop.hbase.mob.MobUtils;
-import org.apache.hadoop.hbase.util.FSUtils;
-
-/**
- * A mob compactor to directly compact the mob files.
- */
-@InterfaceAudience.Private
-public abstract class MobCompactor {
-
-  protected FileSystem fs;
-  protected Configuration conf;
-  protected TableName tableName;
-  protected ColumnFamilyDescriptor column;
-
-  protected Path mobTableDir;
-  protected Path mobFamilyDir;
-  protected ExecutorService pool;
-
-  public MobCompactor(Configuration conf, FileSystem fs, TableName tableName,
-                      ColumnFamilyDescriptor column, ExecutorService pool) {
-    this.conf = conf;
-    this.fs = fs;
-    this.tableName = tableName;
-    this.column = column;
-    this.pool = pool;
-    mobTableDir = FSUtils.getTableDir(MobUtils.getMobHome(conf), tableName);
-    mobFamilyDir = MobUtils.getMobFamilyPath(conf, tableName, column.getNameAsString());
-  }
-
-  /**
-   * Compacts the mob files for the current column family.
-   * @return The paths of new mob files generated in the compaction.
-   * @throws IOException
-   */
-  public List<Path> compact() throws IOException {
-    return compact(false);
-  }
-
-  /**
-   * Compacts the mob files by compaction type for the current column family.
-   * @param allFiles Whether add all mob files into the compaction.
-   * @return The paths of new mob files generated in the compaction.
-   * @throws IOException
-   */
-  public List<Path> compact(boolean allFiles) throws IOException {
-    return compact(Arrays.asList(fs.listStatus(mobFamilyDir)), allFiles);
-  }
-
-  /**
-   * Compacts the candidate mob files.
-   * @param files The candidate mob files.
-   * @param allFiles Whether add all mob files into the compaction.
-   * @return The paths of new mob files generated in the compaction.
-   * @throws IOException
-   */
-  public abstract List<Path> compact(List<FileStatus> files, boolean allFiles)
-    throws IOException;
-}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactionRequest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactionRequest.java
deleted file mode 100644
index ab917a2..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactionRequest.java
+++ /dev/null
@@ -1,333 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.mob.compactions;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.regionserver.HStoreFile;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.yetus.audience.InterfaceAudience;
-
-/**
- * An implementation of {@link MobCompactionRequest} that is used in
- * {@link PartitionedMobCompactor}.
- * The mob files that have the same start key and date in their names belong to
- * the same partition.
- */
-@InterfaceAudience.Private
-public class PartitionedMobCompactionRequest extends MobCompactionRequest {
-
-  protected List<CompactionDelPartition> delPartitions;
-  protected Collection<CompactionPartition> compactionPartitions;
-
-  public PartitionedMobCompactionRequest(Collection<CompactionPartition> compactionPartitions,
-    List<CompactionDelPartition> delPartitions) {
-    this.selectionTime = EnvironmentEdgeManager.currentTime();
-    this.compactionPartitions = compactionPartitions;
-    this.delPartitions = delPartitions;
-  }
-
-  /**
-   * Gets the compaction partitions.
-   * @return The compaction partitions.
-   */
-  public Collection<CompactionPartition> getCompactionPartitions() {
-    return this.compactionPartitions;
-  }
-
-  /**
-   * Gets the del files.
-   * @return The del files.
-   */
-  public List<CompactionDelPartition> getDelPartitions() {
-    return this.delPartitions;
-  }
-
-  /**
-   * The partition in the mob compaction.
-   * The mob files that have the same start key and date in their names belong to
-   * the same partition.
-   */
-  protected static class CompactionPartition {
-    private List<FileStatus> files = new ArrayList<>();
-    private CompactionPartitionId partitionId;
-
-    // The startKey and endKey of this partition, both are inclusive.
-    private byte[] startKey;
-    private byte[] endKey;
-
-    public CompactionPartition(CompactionPartitionId partitionId) {
-      this.partitionId = partitionId;
-    }
-
-    public CompactionPartitionId getPartitionId() {
-      return this.partitionId;
-    }
-
-    public void addFile(FileStatus file) {
-      files.add(file);
-    }
-
-    public List<FileStatus> listFiles() {
-      return Collections.unmodifiableList(files);
-    }
-
-    public int getFileCount () {
-      return files.size();
-    }
-
-    public byte[] getStartKey() {
-      return startKey;
-    }
-
-    /**
-     * Set start key of this partition, only if the input startKey is less than
-     * the current start key.
-     */
-    public void setStartKey(final byte[] startKey) {
-      if ((this.startKey == null) || (Bytes.compareTo(startKey, this.startKey) < 0)) {
-        this.startKey = startKey;
-      }
-    }
-
-    public byte[] getEndKey() {
-      return endKey;
-    }
-
-    /**
-     * Set end key of this partition, only if the input endKey is greater than
-     * the current end key.
-     */
-    public void setEndKey(final byte[] endKey) {
-      if ((this.endKey == null) || (Bytes.compareTo(endKey, this.endKey) > 0)) {
-        this.endKey = endKey;
-      }
-    }
-  }
-
-  /**
-   * The partition id that consists of start key and date of the mob file name.
-   */
-  public static class CompactionPartitionId {
-    private String startKey;
-    private String date;
-    private String latestDate;
-    private long threshold;
-
-    public CompactionPartitionId() {
-      // initialize these fields to empty string
-      this.startKey = "";
-      this.date = "";
-      this.latestDate = "";
-      this.threshold = 0;
-    }
-
-    public CompactionPartitionId(String startKey, String date) {
-      if (startKey == null || date == null) {
-        throw new IllegalArgumentException("Neither of start key and date could be null");
-      }
-      this.startKey = startKey;
-      this.date = date;
-      this.latestDate = "";
-      this.threshold = 0;
-    }
-
-    public void setThreshold (final long threshold) {
-      this.threshold = threshold;
-    }
-
-    public long getThreshold () {
-      return this.threshold;
-    }
-
-    public String getStartKey() {
-      return this.startKey;
-    }
-
-    public void setStartKey(final String startKey) {
-      this.startKey = startKey;
-    }
-
-    public String getDate() {
-      return this.date;
-    }
-
-    public void setDate(final String date) {
-      this.date = date;
-    }
-
-    public String getLatestDate () { return this.latestDate; }
-
-    public void updateLatestDate(final String latestDate) {
-      if (this.latestDate.compareTo(latestDate) < 0) {
-        this.latestDate = latestDate;
-      }
-    }
-
-    @Override
-    public int hashCode() {
-      int result = 17;
-      result = 31 * result + startKey.hashCode();
-      result = 31 * result + date.hashCode();
-      return result;
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-      if (this == obj) {
-        return true;
-      }
-      if (!(obj instanceof CompactionPartitionId)) {
-        return false;
-      }
-      CompactionPartitionId another = (CompactionPartitionId) obj;
-      if (!this.startKey.equals(another.startKey)) {
-        return false;
-      }
-      if (!this.date.equals(another.date)) {
-        return false;
-      }
-      return true;
-    }
-
-    @Override
-    public String toString() {
-      return new StringBuilder(startKey).append(date).toString();
-    }
-  }
-
-  /**
-   * The delete file partition in the mob compaction.
-   * The delete partition is defined as [startKey, endKey] pair.
-   * The mob delete files that have the same start key and end key belong to
-   * the same partition.
-   */
-  protected static class CompactionDelPartition {
-    private List<Path> delFiles = new ArrayList<Path>();
-    private List<HStoreFile> storeFiles = new ArrayList<>();
-    private CompactionDelPartitionId id;
-
-    public CompactionDelPartition(CompactionDelPartitionId id) {
-      this.id = id;
-    }
-
-    public CompactionDelPartitionId getId() {
-      return this.id;
-    }
-
-    void addDelFile(FileStatus file) {
-      delFiles.add(file.getPath());
-    }
-    public void addStoreFile(HStoreFile file) {
-      storeFiles.add(file);
-    }
-
-    public List<HStoreFile> getStoreFiles() {
-      return storeFiles;
-    }
-
-    List<Path> listDelFiles() {
-      return Collections.unmodifiableList(delFiles);
-    }
-
-    void addDelFileList(final Collection<Path> list) {
-      delFiles.addAll(list);
-    }
-
-    int getDelFileCount () {
-      return delFiles.size();
-    }
-
-    void cleanDelFiles() {
-      delFiles.clear();
-    }
-  }
-
-  /**
-   * The delete partition id that consists of start key and end key
-   */
-  public static class CompactionDelPartitionId implements Comparable<CompactionDelPartitionId> {
-    private byte[] startKey;
-    private byte[] endKey;
-
-    public CompactionDelPartitionId() {
-    }
-
-    public CompactionDelPartitionId(final byte[] startKey, final byte[] endKey) {
-      this.startKey = startKey;
-      this.endKey = endKey;
-    }
-
-    public byte[] getStartKey() {
-      return this.startKey;
-    }
-    public void setStartKey(final byte[] startKey) {
-      this.startKey = startKey;
-    }
-
-    public byte[] getEndKey() {
-      return this.endKey;
-    }
-    public void setEndKey(final byte[] endKey) {
-      this.endKey = endKey;
-    }
-
-    @Override
-    public int compareTo(CompactionDelPartitionId o) {
-      /*
-       * 1). Compare the start key, if the k1 < k2, then k1 is less
-       * 2). If start Key is same, check endKey, k1 < k2, k1 is less
-       *     If both are same, then they are equal.
-       */
-      int result = Bytes.compareTo(this.startKey, o.getStartKey());
-      if (result != 0) {
-        return result;
-      }
-
-      return Bytes.compareTo(this.endKey, o.getEndKey());
-    }
-
-    @Override
-    public int hashCode() {
-      int result = 17;
-      result = 31 * result + java.util.Arrays.hashCode(startKey);
-      result = 31 * result + java.util.Arrays.hashCode(endKey);
-      return result;
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-      if (this == obj) {
-        return true;
-      }
-      if (!(obj instanceof CompactionDelPartitionId)) {
-        return false;
-      }
-      CompactionDelPartitionId another = (CompactionDelPartitionId) obj;
-
-      return (this.compareTo(another) == 0);
-    }
-  }
-}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java
deleted file mode 100644
index 669bffc..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java
+++ /dev/null
@@ -1,949 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.mob.compactions;
-
-import static org.apache.hadoop.hbase.regionserver.HStoreFile.BULKLOAD_TIME_KEY;
-import static org.apache.hadoop.hbase.regionserver.HStoreFile.MOB_CELLS_COUNT;
-import static org.apache.hadoop.hbase.regionserver.HStoreFile.SKIP_RESET_SEQ_ID;
-
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Calendar;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.NavigableMap;
-import java.util.Objects;
-import java.util.TreeMap;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Future;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.ArrayBackedTag;
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.CellComparator;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.Tag;
-import org.apache.hadoop.hbase.TagType;
-import org.apache.hadoop.hbase.TagUtil;
-import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.client.MobCompactPartitionPolicy;
-import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.io.HFileLink;
-import org.apache.hadoop.hbase.io.crypto.Encryption;
-import org.apache.hadoop.hbase.io.hfile.CacheConfig;
-import org.apache.hadoop.hbase.io.hfile.HFile;
-import org.apache.hadoop.hbase.io.hfile.HFile.Reader;
-import org.apache.hadoop.hbase.mob.MobConstants;
-import org.apache.hadoop.hbase.mob.MobFileName;
-import org.apache.hadoop.hbase.mob.MobUtils;
-import org.apache.hadoop.hbase.mob.compactions.MobCompactionRequest.CompactionType;
-import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionDelPartition;
-import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionDelPartitionId;
-import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionPartition;
-import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionPartitionId;
-import org.apache.hadoop.hbase.regionserver.BloomType;
-import org.apache.hadoop.hbase.regionserver.HStore;
-import org.apache.hadoop.hbase.regionserver.HStoreFile;
-import org.apache.hadoop.hbase.regionserver.ScanInfo;
-import org.apache.hadoop.hbase.regionserver.ScanType;
-import org.apache.hadoop.hbase.regionserver.ScannerContext;
-import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
-import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
-import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
-import org.apache.hadoop.hbase.regionserver.StoreScanner;
-import org.apache.hadoop.hbase.security.EncryptionUtil;
-import org.apache.hadoop.hbase.tool.BulkLoadHFiles;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.Pair;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
-/**
- * An implementation of {@link MobCompactor} that compacts the mob files in partitions.
- */
-@InterfaceAudience.Private
-public class PartitionedMobCompactor extends MobCompactor {
-
-  private static final Logger LOG = LoggerFactory.getLogger(PartitionedMobCompactor.class);
-  protected long mergeableSize;
-  protected int delFileMaxCount;
-  /** The number of files compacted in a batch */
-  protected int compactionBatchSize;
-  protected int compactionKVMax;
-
-  private final Path tempPath;
-  private final Path bulkloadPath;
-  private final CacheConfig compactionCacheConfig;
-  private final byte[] refCellTags;
-  private Encryption.Context cryptoContext = Encryption.Context.NONE;
-
-  public PartitionedMobCompactor(Configuration conf, FileSystem fs, TableName tableName,
-                                 ColumnFamilyDescriptor column, ExecutorService pool) throws IOException {
-    super(conf, fs, tableName, column, pool);
-    mergeableSize = conf.getLong(MobConstants.MOB_COMPACTION_MERGEABLE_THRESHOLD,
-      MobConstants.DEFAULT_MOB_COMPACTION_MERGEABLE_THRESHOLD);
-    delFileMaxCount = conf.getInt(MobConstants.MOB_DELFILE_MAX_COUNT,
-      MobConstants.DEFAULT_MOB_DELFILE_MAX_COUNT);
-    // default is 100
-    compactionBatchSize = conf.getInt(MobConstants.MOB_COMPACTION_BATCH_SIZE,
-      MobConstants.DEFAULT_MOB_COMPACTION_BATCH_SIZE);
-    tempPath = new Path(MobUtils.getMobHome(conf), MobConstants.TEMP_DIR_NAME);
-    bulkloadPath = new Path(tempPath, new Path(MobConstants.BULKLOAD_DIR_NAME, new Path(
-      tableName.getNamespaceAsString(), tableName.getQualifierAsString())));
-    compactionKVMax = this.conf.getInt(HConstants.COMPACTION_KV_MAX,
-      HConstants.COMPACTION_KV_MAX_DEFAULT);
-    Configuration copyOfConf = new Configuration(conf);
-    copyOfConf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0f);
-    compactionCacheConfig = new CacheConfig(copyOfConf);
-    List<Tag> tags = new ArrayList<>(2);
-    tags.add(MobConstants.MOB_REF_TAG);
-    Tag tableNameTag = new ArrayBackedTag(TagType.MOB_TABLE_NAME_TAG_TYPE, tableName.getName());
-    tags.add(tableNameTag);
-    this.refCellTags = TagUtil.fromList(tags);
-    cryptoContext = EncryptionUtil.createEncryptionContext(copyOfConf, column);
-  }
-
-  @Override
-  public List<Path> compact(List<FileStatus> files, boolean allFiles) throws IOException {
-    if (files == null || files.isEmpty()) {
-      LOG.info("No candidate mob files");
-      return null;
-    }
-    LOG.info("is allFiles: " + allFiles);
-
-    // find the files to compact.
-    PartitionedMobCompactionRequest request = select(files, allFiles);
-    // compact the files.
-    return performCompaction(request);
-  }
-
-  /**
-   * Selects the compacted mob/del files.
-   * Iterates the candidates to find out all the del files and small mob files.
-   * @param candidates All the candidates.
-   * @param allFiles Whether add all mob files into the compaction.
-   * @return A compaction request.
-   * @throws IOException if IO failure is encountered
-   */
-  protected PartitionedMobCompactionRequest select(List<FileStatus> candidates,
-    boolean allFiles) throws IOException {
-    final Map<CompactionPartitionId, CompactionPartition> filesToCompact = new HashMap<>();
-    final CompactionPartitionId id = new CompactionPartitionId();
-    final NavigableMap<CompactionDelPartitionId, CompactionDelPartition> delFilesToCompact = new TreeMap<>();
-    final CompactionDelPartitionId delId = new CompactionDelPartitionId();
-    final ArrayList<CompactionDelPartition> allDelPartitions = new ArrayList<>();
-    int selectedFileCount = 0;
-    int irrelevantFileCount = 0;
-    int totalDelFiles = 0;
-    MobCompactPartitionPolicy policy = column.getMobCompactPartitionPolicy();
-
-    Calendar calendar =  Calendar.getInstance();
-    Date currentDate = new Date();
-    Date firstDayOfCurrentMonth = null;
-    Date firstDayOfCurrentWeek = null;
-
-    if (policy == MobCompactPartitionPolicy.MONTHLY) {
-      firstDayOfCurrentMonth = MobUtils.getFirstDayOfMonth(calendar, currentDate);
-      firstDayOfCurrentWeek = MobUtils.getFirstDayOfWeek(calendar, currentDate);
-    } else if (policy == MobCompactPartitionPolicy.WEEKLY) {
-      firstDayOfCurrentWeek = MobUtils.getFirstDayOfWeek(calendar, currentDate);
-    }
-
-    // We check if there is any del files so the logic can be optimized for the following processing
-    // First step is to check if there is any delete files. If there is any delete files,
-    // For each Partition, it needs to read its startKey and endKey from files.
-    // If there is no delete file, there is no need to read startKey and endKey from files, this
-    // is an optimization.
-    boolean withDelFiles = false;
-    for (FileStatus file : candidates) {
-      if (!file.isFile()) {
-        continue;
-      }
-      // group the del files and small files.
-      FileStatus linkedFile = file;
-      if (HFileLink.isHFileLink(file.getPath())) {
-        HFileLink link = HFileLink.buildFromHFileLinkPattern(conf, file.getPath());
-        linkedFile = getLinkedFileStatus(link);
-        if (linkedFile == null) {
-          continue;
-        }
-      }
-      if (StoreFileInfo.isDelFile(linkedFile.getPath())) {
-        withDelFiles = true;
-        break;
-      }
-    }
-
-    for (FileStatus file : candidates) {
-      if (!file.isFile()) {
-        irrelevantFileCount++;
-        continue;
-      }
-      // group the del files and small files.
-      FileStatus linkedFile = file;
-      if (HFileLink.isHFileLink(file.getPath())) {
-        HFileLink link = HFileLink.buildFromHFileLinkPattern(conf, file.getPath());
-        linkedFile = getLinkedFileStatus(link);
-        if (linkedFile == null) {
-          // If the linked file cannot be found, regard it as an irrelevantFileCount file
-          irrelevantFileCount++;
-          continue;
-        }
-      }
-      if (withDelFiles && StoreFileInfo.isDelFile(linkedFile.getPath())) {
-        // File in the Del Partition List
-
-        // Get delId from the file
-        try (Reader reader = HFile.createReader(fs, linkedFile.getPath(), conf)) {
-          delId.setStartKey(reader.getFirstRowKey().get());
-          delId.setEndKey(reader.getLastRowKey().get());
-        }
-        CompactionDelPartition delPartition = delFilesToCompact.get(delId);
-        if (delPartition == null) {
-          CompactionDelPartitionId newDelId =
-              new CompactionDelPartitionId(delId.getStartKey(), delId.getEndKey());
-          delPartition = new CompactionDelPartition(newDelId);
-          delFilesToCompact.put(newDelId, delPartition);
-        }
-        delPartition.addDelFile(file);
-        totalDelFiles ++;
-      } else {
-        String fileName = linkedFile.getPath().getName();
-        String date = MobFileName.getDateFromName(fileName);
-        boolean skipCompaction = MobUtils
-            .fillPartitionId(id, firstDayOfCurrentMonth, firstDayOfCurrentWeek, date, policy,
-                calendar, mergeableSize);
-        if (allFiles || (!skipCompaction && (linkedFile.getLen() < id.getThreshold()))) {
-          // add all files if allFiles is true,
-          // otherwise add the small files to the merge pool
-          // filter out files which are not supposed to be compacted with the
-          // current policy
-
-          id.setStartKey(MobFileName.getStartKeyFromName(fileName));
-          CompactionPartition compactionPartition = filesToCompact.get(id);
-          if (compactionPartition == null) {
-            CompactionPartitionId newId = new CompactionPartitionId(id.getStartKey(), id.getDate());
-            compactionPartition = new CompactionPartition(newId);
-            compactionPartition.addFile(file);
-            filesToCompact.put(newId, compactionPartition);
-            newId.updateLatestDate(date);
-          } else {
-            compactionPartition.addFile(file);
-            compactionPartition.getPartitionId().updateLatestDate(date);
-          }
-
-          if (withDelFiles) {
-            // get startKey and endKey from the file and update partition
-            // TODO: is it possible to skip read of most hfiles?
-            try (Reader reader = HFile.createReader(fs, linkedFile.getPath(), conf)) {
-              compactionPartition.setStartKey(reader.getFirstRowKey().get());
-              compactionPartition.setEndKey(reader.getLastRowKey().get());
-            }
-          }
-
-          selectedFileCount++;
-        }
-      }
-    }
-
-    /*
-     * Merge del files so there are only non-overlapped del file lists
-     */
-    for(Map.Entry<CompactionDelPartitionId, CompactionDelPartition> entry : delFilesToCompact.entrySet()) {
-      if (allDelPartitions.size() > 0) {
-        // check if the current key range overlaps the previous one
-        CompactionDelPartition prev = allDelPartitions.get(allDelPartitions.size() - 1);
-        if (Bytes.compareTo(prev.getId().getEndKey(), entry.getKey().getStartKey()) >= 0) {
-          // merge them together
-          prev.getId().setEndKey(entry.getValue().getId().getEndKey());
-          prev.addDelFileList(entry.getValue().listDelFiles());
-
-        } else {
-          allDelPartitions.add(entry.getValue());
-        }
-      } else {
-        allDelPartitions.add(entry.getValue());
-      }
-    }
-
-    PartitionedMobCompactionRequest request = new PartitionedMobCompactionRequest(
-      filesToCompact.values(), allDelPartitions);
-    if (candidates.size() == (totalDelFiles + selectedFileCount + irrelevantFileCount)) {
-      // all the files are selected
-      request.setCompactionType(CompactionType.ALL_FILES);
-    }
-    LOG.info("The compaction type is {}, the request has {} del files, {} selected files, and {} " +
-        "irrelevant files table '{}' and column '{}'", request.getCompactionType(), totalDelFiles,
-        selectedFileCount, irrelevantFileCount, tableName, column.getNameAsString());
-    return request;
-  }
-
-  /**
-   * Performs the compaction on the selected files.
-   * <ol>
-   * <li>Compacts the del files.</li>
-   * <li>Compacts the selected small mob files and all the del files.</li>
-   * <li>If all the candidates are selected, delete the del files.</li>
-   * </ol>
-   * @param request The compaction request.
-   * @return The paths of new mob files generated in the compaction.
-   * @throws IOException if IO failure is encountered
-   */
-  protected List<Path> performCompaction(PartitionedMobCompactionRequest request)
-    throws IOException {
-
-    // merge the del files, it is per del partition
-    for (CompactionDelPartition delPartition : request.getDelPartitions()) {
-      if (delPartition.getDelFileCount() <= 1) continue;
-      List<Path> newDelPaths = compactDelFiles(request, delPartition.listDelFiles());
-      delPartition.cleanDelFiles();
-      delPartition.addDelFileList(newDelPaths);
-    }
-
-    List<Path> paths = null;
-    int totalDelFileCount = 0;
-    try {
-      for (CompactionDelPartition delPartition : request.getDelPartitions()) {
-        for (Path newDelPath : delPartition.listDelFiles()) {
-          HStoreFile sf =
-              new HStoreFile(fs, newDelPath, conf, compactionCacheConfig, BloomType.NONE, true);
-          // pre-create reader of a del file to avoid race condition when opening the reader in each
-          // partition.
-          sf.initReader();
-          delPartition.addStoreFile(sf);
-          totalDelFileCount++;
-        }
-      }
-      LOG.info("After merging, there are {} del files. table='{}' column='{}'", totalDelFileCount,
-          tableName, column.getNameAsString());
-      // compact the mob files by partitions.
-      paths = compactMobFiles(request);
-      LOG.info("After compaction, there are {} mob files. table='{}' column='{}'", paths.size(),
-          tableName, column.getNameAsString());
-    } finally {
-      for (CompactionDelPartition delPartition : request.getDelPartitions()) {
-        closeStoreFileReaders(delPartition.getStoreFiles());
-      }
-    }
-
-    // archive the del files if all the mob files are selected.
-    if (request.type == CompactionType.ALL_FILES && !request.getDelPartitions().isEmpty()) {
-      LOG.info("After a mob compaction with all files selected, archiving the del files for " +
-          "table='{}' and column='{}'", tableName, column.getNameAsString());
-      for (CompactionDelPartition delPartition : request.getDelPartitions()) {
-        LOG.info(Objects.toString(delPartition.listDelFiles()));
-        try {
-          MobUtils.removeMobFiles(conf, fs, tableName, mobTableDir, column.getName(),
-            delPartition.getStoreFiles());
-        } catch (IOException e) {
-          LOG.error("Failed to archive the del files {} for partition {} table='{}' and " +
-              "column='{}'", delPartition.getStoreFiles(), delPartition.getId(), tableName,
-              column.getNameAsString(), e);
-        }
-      }
-    }
-    return paths;
-  }
-
-  static class DelPartitionComparator implements Comparator<CompactionDelPartition> {
-    private boolean compareStartKey;
-
-    DelPartitionComparator(boolean compareStartKey) {
-      this.compareStartKey = compareStartKey;
-    }
-
-    public boolean getCompareStartKey() {
-      return this.compareStartKey;
-    }
-
-    public void setCompareStartKey(final boolean compareStartKey) {
-      this.compareStartKey = compareStartKey;
-    }
-
-    @Override
-    public int compare(CompactionDelPartition o1, CompactionDelPartition o2) {
-
-      if (compareStartKey) {
-        return Bytes.compareTo(o1.getId().getStartKey(), o2.getId().getStartKey());
-      } else {
-        return Bytes.compareTo(o1.getId().getEndKey(), o2.getId().getEndKey());
-      }
-    }
-  }
-
-  @VisibleForTesting
-  List<HStoreFile> getListOfDelFilesForPartition(final CompactionPartition partition,
-      final List<CompactionDelPartition> delPartitions) {
-    // Binary search for startKey and endKey
-
-    List<HStoreFile> result = new ArrayList<>();
-
-    DelPartitionComparator comparator = new DelPartitionComparator(false);
-    CompactionDelPartitionId id = new CompactionDelPartitionId(null, partition.getStartKey());
-    CompactionDelPartition target = new CompactionDelPartition(id);
-    int start = Collections.binarySearch(delPartitions, target, comparator);
-
-    // Get the start index for partition
-    if (start < 0) {
-      // Calculate the insert point
-      start = (start + 1) * (-1);
-      if (start == delPartitions.size()) {
-        // no overlap
-        return result;
-      } else {
-        // Check another case which has no overlap
-        if (Bytes.compareTo(partition.getEndKey(), delPartitions.get(start).getId().getStartKey()) < 0) {
-          return result;
-        }
-      }
-    }
-
-    // Search for end index for the partition
-    comparator.setCompareStartKey(true);
-    id.setStartKey(partition.getEndKey());
-    int end = Collections.binarySearch(delPartitions, target, comparator);
-
-    if (end < 0) {
-      end = (end + 1) * (-1);
-      if (end == 0) {
-        return result;
-      } else {
-        --end;
-        if (Bytes.compareTo(partition.getStartKey(), delPartitions.get(end).getId().getEndKey()) > 0) {
-          return result;
-        }
-      }
-    }
-
-    for (int i = start; i <= end; ++i) {
-        result.addAll(delPartitions.get(i).getStoreFiles());
-    }
-
-    return result;
-  }
-
-  /**
-   * Compacts the selected small mob files and all the del files.
-   * @param request The compaction request.
-   * @return The paths of new mob files after compactions.
-   * @throws IOException if IO failure is encountered
-   */
-  protected List<Path> compactMobFiles(final PartitionedMobCompactionRequest request)
-      throws IOException {
-    Collection<CompactionPartition> partitions = request.compactionPartitions;
-    if (partitions == null || partitions.isEmpty()) {
-      LOG.info("No partitions of mob files in table='{}' and column='{}'", tableName,
-          column.getNameAsString());
-      return Collections.emptyList();
-    }
-    List<Path> paths = new ArrayList<>();
-    final Connection c = ConnectionFactory.createConnection(conf);
-    final Table table = c.getTable(tableName);
-
-    try {
-      Map<CompactionPartitionId, Future<List<Path>>> results = new HashMap<>();
-      // compact the mob files by partitions in parallel.
-      for (final CompactionPartition partition : partitions) {
-
-        // How to efficiently come up a list of delFiles for one partition?
-        // Search the delPartitions and collect all the delFiles for the partition
-        // One optimization can do is that if there is no del file, we do not need to
-        // come up with startKey/endKey.
-        List<HStoreFile> delFiles = getListOfDelFilesForPartition(partition,
-            request.getDelPartitions());
-
-        results.put(partition.getPartitionId(), pool.submit(new Callable<List<Path>>() {
-          @Override
-          public List<Path> call() throws Exception {
-            LOG.info("Compacting mob files for partition {} for table='{}' and column='{}'",
-                partition.getPartitionId(), tableName, column.getNameAsString());
-            return compactMobFilePartition(request, partition, delFiles, c, table);
-          }
-        }));
-      }
-      // compact the partitions in parallel.
-      List<CompactionPartitionId> failedPartitions = new ArrayList<>();
-      for (Entry<CompactionPartitionId, Future<List<Path>>> result : results.entrySet()) {
-        try {
-          paths.addAll(result.getValue().get());
-        } catch (Exception e) {
-          // just log the error
-          LOG.error("Failed to compact the partition {} for table='{}' and column='{}'",
-              result.getKey(), tableName, column.getNameAsString(), e);
-          failedPartitions.add(result.getKey());
-        }
-      }
-      if (!failedPartitions.isEmpty()) {
-        // if any partition fails in the compaction, directly throw an exception.
-        throw new IOException("Failed to compact the partitions " + failedPartitions +
-            " for table='" + tableName + "' column='" + column.getNameAsString() + "'");
-      }
-    } finally {
-      try {
-        table.close();
-      } catch (IOException e) {
-        LOG.error("Failed to close the Table", e);
-      }
-    }
-    return paths;
-  }
-
-  /**
-   * Compacts a partition of selected small mob files and all the del files.
-   * @param request The compaction request.
-   * @param partition A compaction partition.
-   * @param delFiles The del files.
-   * @param connection The connection to use.
-   * @param table The current table.
-   * @return The paths of new mob files after compactions.
-   * @throws IOException if IO failure is encountered
-   */
-  private List<Path> compactMobFilePartition(PartitionedMobCompactionRequest request,
-                                             CompactionPartition partition,
-                                             List<HStoreFile> delFiles,
-                                             Connection connection,
-                                             Table table) throws IOException {
-    if (MobUtils.isMobFileExpired(column, EnvironmentEdgeManager.currentTime(),
-      partition.getPartitionId().getDate())) {
-      // If the files in the partition are expired, do not compact them and directly
-      // return an empty list.
-      return Collections.emptyList();
-    }
-    List<Path> newFiles = new ArrayList<>();
-    List<FileStatus> files = partition.listFiles();
-    int offset = 0;
-    Path bulkloadPathOfPartition = new Path(bulkloadPath, partition.getPartitionId().toString());
-    Path bulkloadColumnPath = new Path(bulkloadPathOfPartition, column.getNameAsString());
-    while (offset < files.size()) {
-      int batch = compactionBatchSize;
-      if (files.size() - offset < compactionBatchSize) {
-        batch = files.size() - offset;
-      }
-      if (batch == 1 && delFiles.isEmpty()) {
-        // only one file left and no del files, do not compact it,
-        // and directly add it to the new files.
-        newFiles.add(files.get(offset).getPath());
-        offset++;
-        continue;
-      }
-      // clean the bulkload directory to avoid loading old files.
-      fs.delete(bulkloadPathOfPartition, true);
-      // add the selected mob files and del files into filesToCompact
-      List<HStoreFile> filesToCompact = new ArrayList<>();
-      for (int i = offset; i < batch + offset; i++) {
-        HStoreFile sf = new HStoreFile(fs, files.get(i).getPath(), conf, compactionCacheConfig,
-            BloomType.NONE, true);
-        filesToCompact.add(sf);
-      }
-      filesToCompact.addAll(delFiles);
-      // compact the mob files in a batch.
-      compactMobFilesInBatch(request, partition, connection, table, filesToCompact, batch,
-        bulkloadPathOfPartition, bulkloadColumnPath, newFiles);
-      // move to the next batch.
-      offset += batch;
-    }
-    LOG.info("Compaction is finished. The number of mob files is changed from {} to {} for " +
-        "partition={} for table='{}' and column='{}'", files.size(), newFiles.size(),
-        partition.getPartitionId(), tableName, column.getNameAsString());
-    return newFiles;
-  }
-
-  /**
-   * Closes the readers of store files.
-   * @param storeFiles The store files to be closed.
-   */
-  private void closeStoreFileReaders(List<HStoreFile> storeFiles) {
-    for (HStoreFile storeFile : storeFiles) {
-      try {
-        storeFile.closeStoreFile(true);
-      } catch (IOException e) {
-        LOG.warn("Failed to close the reader on store file " + storeFile.getPath(), e);
-      }
-    }
-  }
-
-  /**
-   * Compacts a partition of selected small mob files and all the del files in a batch.
-   * @param request The compaction request.
-   * @param partition A compaction partition.
-   * @param connection To use for transport
-   * @param table The current table.
-   * @param filesToCompact The files to be compacted.
-   * @param batch The number of mob files to be compacted in a batch.
-   * @param bulkloadPathOfPartition The directory where the bulkload column of the current
-   *   partition is saved.
-   * @param bulkloadColumnPath The directory where the bulkload files of current partition
-   *   are saved.
-   * @param newFiles The paths of new mob files after compactions.
-   * @throws IOException if IO failure is encountered
-   */
-  private void compactMobFilesInBatch(PartitionedMobCompactionRequest request,
-                                      CompactionPartition partition,
-                                      Connection connection, Table table,
-                                      List<HStoreFile> filesToCompact, int batch,
-                                      Path bulkloadPathOfPartition, Path bulkloadColumnPath,
-                                      List<Path> newFiles)
-      throws IOException {
-    // open scanner to the selected mob files and del files.
-    StoreScanner scanner = createScanner(filesToCompact, ScanType.COMPACT_DROP_DELETES);
-    // the mob files to be compacted, not include the del files.
-    List<HStoreFile> mobFilesToCompact = filesToCompact.subList(0, batch);
-    // Pair(maxSeqId, cellsCount)
-    Pair<Long, Long> fileInfo = getFileInfo(mobFilesToCompact);
-    // open writers for the mob files and new ref store files.
-    StoreFileWriter writer = null;
-    StoreFileWriter refFileWriter = null;
-    Path filePath = null;
-    long mobCells = 0;
-    boolean cleanupTmpMobFile = false;
-    boolean cleanupBulkloadDirOfPartition = false;
-    boolean cleanupCommittedMobFile = false;
-    boolean closeReaders= true;
-
-    try {
-      try {
-        writer = MobUtils
-            .createWriter(conf, fs, column, partition.getPartitionId().getLatestDate(), tempPath,
-                Long.MAX_VALUE, column.getCompactionCompressionType(),
-                partition.getPartitionId().getStartKey(), compactionCacheConfig, cryptoContext,
-                true);
-        cleanupTmpMobFile = true;
-        filePath = writer.getPath();
-        byte[] fileName = Bytes.toBytes(filePath.getName());
-        // create a temp file and open a writer for it in the bulkloadPath
-        refFileWriter = MobUtils.createRefFileWriter(conf, fs, column, bulkloadColumnPath,
-            fileInfo.getSecond().longValue(), compactionCacheConfig, cryptoContext, true);
-        cleanupBulkloadDirOfPartition = true;
-        List<Cell> cells = new ArrayList<>();
-        boolean hasMore;
-        ScannerContext scannerContext =
-            ScannerContext.newBuilder().setBatchLimit(compactionKVMax).build();
-        do {
-          hasMore = scanner.next(cells, scannerContext);
-          for (Cell cell : cells) {
-            // write the mob cell to the mob file.
-            writer.append(cell);
-            // write the new reference cell to the store file.
-            Cell reference = MobUtils.createMobRefCell(cell, fileName, this.refCellTags);
-            refFileWriter.append(reference);
-            mobCells++;
-          }
-          cells.clear();
-        } while (hasMore);
-      } finally {
-        // close the scanner.
-        scanner.close();
-
-        if (cleanupTmpMobFile) {
-          // append metadata to the mob file, and close the mob file writer.
-          closeMobFileWriter(writer, fileInfo.getFirst(), mobCells);
-        }
-
-        if (cleanupBulkloadDirOfPartition) {
-          // append metadata and bulkload info to the ref mob file, and close the writer.
-          closeRefFileWriter(refFileWriter, fileInfo.getFirst(), request.selectionTime);
-        }
-      }
-
-      if (mobCells > 0) {
-        // commit mob file
-        MobUtils.commitFile(conf, fs, filePath, mobFamilyDir, compactionCacheConfig);
-        cleanupTmpMobFile = false;
-        cleanupCommittedMobFile = true;
-        // bulkload the ref file
-        LOG.info("start MOB ref bulkload for partition {} table='{}' column='{}'",
-            partition.getPartitionId(), tableName, column.getNameAsString());
-        bulkloadRefFile(table.getName(), bulkloadPathOfPartition, filePath.getName());
-        cleanupCommittedMobFile = false;
-        LOG.info("end MOB ref bulkload for partition {} table='{}' column='{}'",
-            partition.getPartitionId(), tableName, column.getNameAsString());
-        newFiles.add(new Path(mobFamilyDir, filePath.getName()));
-      }
-
-      // archive the old mob files, do not archive the del files.
-      try {
-        closeStoreFileReaders(mobFilesToCompact);
-        closeReaders = false;
-        MobUtils.removeMobFiles(conf, fs, tableName, mobTableDir, column.getName(), mobFilesToCompact);
-      } catch (IOException e) {
-        LOG.error("Failed to archive the files " + mobFilesToCompact, e);
-      }
-    } finally {
-      if (closeReaders) {
-        closeStoreFileReaders(mobFilesToCompact);
-      }
-
-      if (cleanupTmpMobFile) {
-        deletePath(filePath);
-      }
-
-      if (cleanupBulkloadDirOfPartition) {
-        // delete the bulkload files in bulkloadPath
-        deletePath(bulkloadPathOfPartition);
-      }
-
-      if (cleanupCommittedMobFile) {
-        LOG.error("failed MOB ref bulkload for partition {} table='{}' column='{}'",
-            partition.getPartitionId(), tableName, column.getNameAsString());
-        MobUtils.removeMobFiles(conf, fs, tableName, mobTableDir, column.getName(),
-            Collections.singletonList(new HStoreFile(fs, new Path(mobFamilyDir, filePath.getName()),
-            conf, compactionCacheConfig, BloomType.NONE, true)));
-      }
-    }
-  }
-
-  /**
-   * Compacts the del files in batches which avoids opening too many files.
-   * @param request The compaction request.
-   * @param delFilePaths Del file paths to compact
-   * @return The paths of new del files after merging or the original files if no merging
-   *         is necessary.
-   * @throws IOException if IO failure is encountered
-   */
-  protected List<Path> compactDelFiles(PartitionedMobCompactionRequest request,
-    List<Path> delFilePaths) throws IOException {
-    if (delFilePaths.size() <= delFileMaxCount) {
-      return delFilePaths;
-    }
-    // when there are more del files than the number that is allowed, merge it firstly.
-    int offset = 0;
-    List<Path> paths = new ArrayList<>();
-    while (offset < delFilePaths.size()) {
-      // get the batch
-      int batch = compactionBatchSize;
-      if (delFilePaths.size() - offset < compactionBatchSize) {
-        batch = delFilePaths.size() - offset;
-      }
-      List<HStoreFile> batchedDelFiles = new ArrayList<>();
-      if (batch == 1) {
-        // only one file left, do not compact it, directly add it to the new files.
-        paths.add(delFilePaths.get(offset));
-        offset++;
-        continue;
-      }
-      for (int i = offset; i < batch + offset; i++) {
-        batchedDelFiles.add(new HStoreFile(fs, delFilePaths.get(i), conf, compactionCacheConfig,
-          BloomType.NONE, true));
-      }
-      // compact the del files in a batch.
-      paths.add(compactDelFilesInBatch(request, batchedDelFiles));
-      // move to the next batch.
-      offset += batch;
-    }
-    return compactDelFiles(request, paths);
-  }
-
-  /**
-   * Compacts the del file in a batch.
-   * @param request The compaction request.
-   * @param delFiles The del files.
-   * @return The path of new del file after merging.
-   * @throws IOException if IO failure is encountered
-   */
-  private Path compactDelFilesInBatch(PartitionedMobCompactionRequest request,
-    List<HStoreFile> delFiles) throws IOException {
-    // create a scanner for the del files.
-    StoreScanner scanner = createScanner(delFiles, ScanType.COMPACT_RETAIN_DELETES);
-    StoreFileWriter writer = null;
-    Path filePath = null;
-    try {
-      writer = MobUtils.createDelFileWriter(conf, fs, column,
-        MobUtils.formatDate(new Date(request.selectionTime)), tempPath, Long.MAX_VALUE,
-        column.getCompactionCompressionType(), HConstants.EMPTY_START_ROW, compactionCacheConfig,
-          cryptoContext);
-      filePath = writer.getPath();
-      List<Cell> cells = new ArrayList<>();
-      boolean hasMore;
-      ScannerContext scannerContext =
-              ScannerContext.newBuilder().setBatchLimit(compactionKVMax).build();
-      do {
-        hasMore = scanner.next(cells, scannerContext);
-        for (Cell cell : cells) {
-          writer.append(cell);
-        }
-        cells.clear();
-      } while (hasMore);
-    } finally {
-      scanner.close();
-      if (writer != null) {
-        try {
-          writer.close();
-        } catch (IOException e) {
-          LOG.error("Failed to close the writer of the file " + filePath, e);
-        }
-      }
-    }
-    // commit the new del file
-    Path path = MobUtils.commitFile(conf, fs, filePath, mobFamilyDir, compactionCacheConfig);
-    // archive the old del files
-    try {
-      MobUtils.removeMobFiles(conf, fs, tableName, mobTableDir, column.getName(), delFiles);
-    } catch (IOException e) {
-      LOG.error("Failed to archive the old del files " + delFiles, e);
-    }
-    return path;
-  }
-
-  /**
-   * Creates a store scanner.
-   * @param filesToCompact The files to be compacted.
-   * @param scanType The scan type.
-   * @return The store scanner.
-   * @throws IOException if IO failure is encountered
-   */
-  private StoreScanner createScanner(List<HStoreFile> filesToCompact, ScanType scanType)
-      throws IOException {
-    List<StoreFileScanner> scanners = StoreFileScanner.getScannersForStoreFiles(filesToCompact,
-      false, true, false, false, HConstants.LATEST_TIMESTAMP);
-    long ttl = HStore.determineTTLFromFamily(column);
-    ScanInfo scanInfo = new ScanInfo(conf, column, ttl, 0, CellComparator.getInstance());
-    return new StoreScanner(scanInfo, scanType, scanners);
-  }
-
-  /**
-   * Bulkloads the current file.
-   * @param tableName The table to load into.
-   * @param bulkloadDirectory The path of bulkload directory.
-   * @param fileName The current file name.
-   * @throws IOException if IO failure is encountered
-   */
-  private void bulkloadRefFile(TableName tableName, Path bulkloadDirectory, String fileName)
-      throws IOException {
-    // bulkload the ref file
-    try {
-      BulkLoadHFiles bulkLoader = BulkLoadHFiles.create(conf);
-      bulkLoader.disableReplication();
-      bulkLoader.bulkLoad(tableName, bulkloadDirectory);
-    } catch (Exception e) {
-      throw new IOException(e);
-    }
-  }
-
-  /**
-   * Closes the mob file writer.
-   * @param writer The mob file writer.
-   * @param maxSeqId Maximum sequence id.
-   * @param mobCellsCount The number of mob cells.
-   * @throws IOException if IO failure is encountered
-   */
-  private void closeMobFileWriter(StoreFileWriter writer, long maxSeqId, long mobCellsCount)
-    throws IOException {
-    if (writer != null) {
-      writer.appendMetadata(maxSeqId, false, mobCellsCount);
-      try {
-        writer.close();
-      } catch (IOException e) {
-        LOG.error("Failed to close the writer of the file " + writer.getPath(), e);
-      }
-    }
-  }
-
-  /**
-   * Closes the ref file writer.
-   * @param writer The ref file writer.
-   * @param maxSeqId Maximum sequence id.
-   * @param bulkloadTime The timestamp at which the bulk load file is created.
-   * @throws IOException if IO failure is encountered
-   */
-  private void closeRefFileWriter(StoreFileWriter writer, long maxSeqId, long bulkloadTime)
-    throws IOException {
-    if (writer != null) {
-      writer.appendMetadata(maxSeqId, false);
-      writer.appendFileInfo(BULKLOAD_TIME_KEY, Bytes.toBytes(bulkloadTime));
-      writer.appendFileInfo(SKIP_RESET_SEQ_ID, Bytes.toBytes(true));
-      try {
-        writer.close();
-      } catch (IOException e) {
-        LOG.error("Failed to close the writer of the ref file " + writer.getPath(), e);
-      }
-    }
-  }
-
-  /**
-   * Gets the max seqId and number of cells of the store files.
-   * @param storeFiles The store files.
-   * @return The pair of the max seqId and number of cells of the store files.
-   * @throws IOException if IO failure is encountered
-   */
-  private Pair<Long, Long> getFileInfo(List<HStoreFile> storeFiles) throws IOException {
-    long maxSeqId = 0;
-    long maxKeyCount = 0;
-    for (HStoreFile sf : storeFiles) {
-      // the readers will be closed later after the merge.
-      maxSeqId = Math.max(maxSeqId, sf.getMaxSequenceId());
-      sf.initReader();
-      byte[] count = sf.getReader().loadFileInfo().get(MOB_CELLS_COUNT);
-      if (count != null) {
-        maxKeyCount += Bytes.toLong(count);
-      }
-    }
-    return new Pair<>(maxSeqId, maxKeyCount);
-  }
-
-  /**
-   * Deletes a file.
-   * @param path The path of the file to be deleted.
-   */
-  private void deletePath(Path path) {
-    LOG.debug("Cleanup, delete path '{}'", path);
-    try {
-      if (path != null) {
-        fs.delete(path, true);
-      }
-    } catch (IOException e) {
-      LOG.error("Failed to delete the file " + path, e);
-    }
-  }
-
-  private FileStatus getLinkedFileStatus(HFileLink link) throws IOException {
-    Path[] locations = link.getLocations();
-    FileStatus file;
-    for (Path location : locations) {
-
-      if (location != null) {
-        try {
-          file = fs.getFileStatus(location);
-          if (file != null) {
-            return file;
-          }
-        }  catch (FileNotFoundException e) {
-        }
-      }
-    }
-    LOG.warn("The file " + link + " links to can not be found");
-    return null;
-  }
-}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java
index b8ea960..8f7b690 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java
@@ -193,28 +193,6 @@ public class HMobStore extends HStore {
       isCompaction);
   }
 
-  /**
-   * Creates the writer for the del file in temp directory.
-   * The del file keeps tracking the delete markers. Its name has a suffix _del,
-   * the format is [0-9a-f]+(_del)?.
-   * @param date The latest date of written cells.
-   * @param maxKeyCount The key count.
-   * @param compression The compression algorithm.
-   * @param startKey The start key.
-   * @return The writer for the del file.
-   * @throws IOException
-   */
-  public StoreFileWriter createDelFileWriterInTmp(Date date, long maxKeyCount,
-      Compression.Algorithm compression, byte[] startKey) throws IOException {
-    if (startKey == null) {
-      startKey = HConstants.EMPTY_START_ROW;
-    }
-    Path path = getTempDir();
-    String suffix = UUID
-        .randomUUID().toString().replaceAll("-", "") + "_del";
-    MobFileName mobFileName = MobFileName.create(startKey, MobUtils.formatDate(date), suffix);
-    return createWriterInTmp(mobFileName, path, maxKeyCount, compression, true);
-  }
 
   /**
    * Creates the writer for the mob file in temp directory.
@@ -231,7 +209,7 @@ public class HMobStore extends HStore {
       Compression.Algorithm compression, byte[] startKey,
       boolean isCompaction) throws IOException {
     MobFileName mobFileName = MobFileName.create(startKey, date, UUID.randomUUID()
-        .toString().replaceAll("-", ""));
+        .toString().replaceAll("-", ""),  region.getRegionInfo().getEncodedName());
     return createWriterInTmp(mobFileName, basePath, maxKeyCount, compression, isCompaction);
   }
 
@@ -265,7 +243,7 @@ public class HMobStore extends HStore {
     }
     Path dstPath = new Path(targetPath, sourceFile.getName());
     validateMobFile(sourceFile);
-    String msg = "Renaming flushed file from " + sourceFile + " to " + dstPath;
+    String msg = " FLUSH Renaming flushed file from " + sourceFile + " to " + dstPath;
     LOG.info(msg);
     Path parent = dstPath.getParent();
     if (!region.getFilesystem().exists(parent)) {
@@ -310,6 +288,20 @@ public class HMobStore extends HStore {
   }
 
   /**
+   * Reads the cell from the mob file with readEmptyValueOnMobCellMiss
+   * @param reference The cell found in the HBase, its value is a path to a mob file.
+   * @param cacheBlocks Whether the scanner should cache blocks.
+   * @param readEmptyValueOnMobCellMiss should return empty mob cell if reference
+   *        can not be resolved.
+   * @return The cell found in the mob file.
+   * @throws IOException
+   */
+  public MobCell resolve(Cell reference, boolean cacheBlocks, boolean readEmptyValueOnMobCellMiss)
+      throws IOException {
+    return resolve(reference, cacheBlocks, -1, readEmptyValueOnMobCellMiss);
+  }
+
+  /**
    * Reads the cell from the mob file.
    * @param reference The cell found in the HBase, its value is a path to a mob file.
    * @param cacheBlocks Whether the scanner should cache blocks.
@@ -520,4 +512,6 @@ public class HMobStore extends HStore {
   public byte[] getRefCellTags() {
     return this.refCellTags;
   }
+
+
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
index d77a472..affd43f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
@@ -1973,7 +1973,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
     finishCompactionRequest(compaction.getRequest());
   }
 
-  private void finishCompactionRequest(CompactionRequestImpl cr) {
+  protected void finishCompactionRequest(CompactionRequestImpl cr) {
     this.region.reportCompactionRequestEnd(cr.isMajor(), cr.getFiles().size(), cr.getSize());
     if (cr.isOffPeak()) {
       offPeakCompactionTracker.set(false);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java
index fa152c5..0f227be 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java
@@ -107,6 +107,12 @@ public class HStoreFile implements StoreFile {
   /** Key for the number of mob cells in metadata */
   public static final byte[] MOB_CELLS_COUNT = Bytes.toBytes("MOB_CELLS_COUNT");
 
+  /** Null data */
+  public static final byte[] NULL_VALUE = new byte[] {0};
+
+  /** Key for the list of MOB file references */
+  public static final byte[] MOB_FILE_REFS = Bytes.toBytes("MOB_FILE_REFS");
+
   /** Meta key set when store file is a result of a bulk load */
   public static final byte[] BULKLOAD_TASK_KEY = Bytes.toBytes("BULKLOAD_SOURCE_TASK");
   public static final byte[] BULKLOAD_TIME_KEY = Bytes.toBytes("BULKLOAD_TIMESTAMP");
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileInfo.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileInfo.java
index 15ed359..b51300a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileInfo.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileInfo.java
@@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.io.hfile.HFileInfo;
 import org.apache.hadoop.hbase.io.hfile.ReaderContext;
 import org.apache.hadoop.hbase.io.hfile.ReaderContext.ReaderType;
 import org.apache.hadoop.hbase.io.hfile.ReaderContextBuilder;
+import org.apache.hadoop.hbase.mob.MobUtils;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
@@ -133,51 +134,50 @@ public class StoreFileInfo {
     this(conf, fs, null, initialPath, primaryReplica);
   }
 
-  private StoreFileInfo(final Configuration conf, final FileSystem fs, final FileStatus fileStatus,
-      final Path initialPath, final boolean primaryReplica) throws IOException {
-    assert fs != null;
-    assert initialPath != null;
-    assert conf != null;
-
-    this.fs = fs;
-    this.conf = conf;
-    this.initialPath = initialPath;
-    this.primaryReplica = primaryReplica;
-    this.noReadahead = this.conf.getBoolean(STORE_FILE_READER_NO_READAHEAD,
-        DEFAULT_STORE_FILE_READER_NO_READAHEAD);
-    Path p = initialPath;
-    if (HFileLink.isHFileLink(p)) {
-      // HFileLink
-      this.reference = null;
-      this.link = HFileLink.buildFromHFileLinkPattern(conf, p);
-      LOG.trace("{} is a link", p);
-    } else if (isReference(p)) {
-      this.reference = Reference.read(fs, p);
-      Path referencePath = getReferredToFile(p);
-      if (HFileLink.isHFileLink(referencePath)) {
-        // HFileLink Reference
-        this.link = HFileLink.buildFromHFileLinkPattern(conf, referencePath);
-      } else {
-        // Reference
-        this.link = null;
-      }
-      LOG.trace("{} is a {} reference to {}", p, reference.getFileRegion(), referencePath);
-    } else if (isHFile(p)) {
-      // HFile
-      if (fileStatus != null) {
-        this.createdTimestamp = fileStatus.getModificationTime();
-        this.size = fileStatus.getLen();
-      } else {
-        FileStatus fStatus = fs.getFileStatus(initialPath);
-        this.createdTimestamp = fStatus.getModificationTime();
-        this.size = fStatus.getLen();
-      }
-      this.reference = null;
-      this.link = null;
-    } else {
-      throw new IOException("path=" + p + " doesn't look like a valid StoreFile");
-    }
-  }
+	private StoreFileInfo(final Configuration conf, final FileSystem fs, final FileStatus fileStatus,
+			final Path initialPath, final boolean primaryReplica) throws IOException {
+		assert fs != null;
+		assert initialPath != null;
+		assert conf != null;
+
+		this.fs = fs;
+		this.conf = conf;
+		this.initialPath = initialPath;
+		this.primaryReplica = primaryReplica;
+		this.noReadahead = this.conf.getBoolean(STORE_FILE_READER_NO_READAHEAD, DEFAULT_STORE_FILE_READER_NO_READAHEAD);
+		Path p = initialPath;
+		if (HFileLink.isHFileLink(p)) {
+			// HFileLink
+			this.reference = null;
+			this.link = HFileLink.buildFromHFileLinkPattern(conf, p);
+			LOG.trace("{} is a link", p);
+		} else if (isReference(p)) {
+			this.reference = Reference.read(fs, p);
+			Path referencePath = getReferredToFile(p);
+			if (HFileLink.isHFileLink(referencePath)) {
+				// HFileLink Reference
+				this.link = HFileLink.buildFromHFileLinkPattern(conf, referencePath);
+			} else {
+				// Reference
+				this.link = null;
+			}
+			LOG.trace("{} is a {} reference to {}", p, reference.getFileRegion(), referencePath);
+		} else if (isHFile(p) || isMobFile(p) || isMobRefFile(p)) {
+			// HFile
+			if (fileStatus != null) {
+				this.createdTimestamp = fileStatus.getModificationTime();
+				this.size = fileStatus.getLen();
+			} else {
+				FileStatus fStatus = fs.getFileStatus(initialPath);
+				this.createdTimestamp = fStatus.getModificationTime();
+				this.size = fStatus.getLen();
+			}
+			this.reference = null;
+			this.link = null;
+		} else {
+			throw new IOException("path=" + p + " doesn't look like a valid StoreFile");
+		}
+	}
 
   /**
    * Create a Store File Info
@@ -442,6 +442,30 @@ public class StoreFileInfo {
     return m.matches() && m.groupCount() > 0;
   }
 
+  public static boolean isMobFile(final Path path) {
+    String fileName = path.getName();
+    String[] parts = fileName.split(MobUtils.SEP);
+    if (parts.length != 2) {
+      return false;
+    }
+    Matcher m = HFILE_NAME_PATTERN.matcher(parts[0]);
+    Matcher mm = HFILE_NAME_PATTERN.matcher(parts[1]);
+    return m.matches() && mm.matches();
+  }
+
+  public static boolean isMobRefFile(final Path path) {
+    String fileName = path.getName();
+    int lastIndex = fileName.lastIndexOf(MobUtils.SEP);
+    if (lastIndex < 0) {
+      return false;
+    }
+    String[] parts = new String[2];
+    parts[0] = fileName.substring(0, lastIndex);
+    parts[1] = fileName.substring(lastIndex + 1);
+    String name = parts[0] + "." + parts[1];
+    Matcher  m = REF_NAME_PATTERN.matcher(name);
+    return m.matches() && m.groupCount() > 1;
+  }
   /**
    * @param path Path to check.
    * @return True if the path has format of a del file.
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java
index 02665a3..2f6a8fb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java
@@ -25,6 +25,8 @@ import static org.apache.hadoop.hbase.regionserver.HStoreFile.EARLIEST_PUT_TS;
 import static org.apache.hadoop.hbase.regionserver.HStoreFile.MAJOR_COMPACTION_KEY;
 import static org.apache.hadoop.hbase.regionserver.HStoreFile.MAX_SEQ_ID_KEY;
 import static org.apache.hadoop.hbase.regionserver.HStoreFile.MOB_CELLS_COUNT;
+import static org.apache.hadoop.hbase.regionserver.HStoreFile.MOB_FILE_REFS;
+import static org.apache.hadoop.hbase.regionserver.HStoreFile.NULL_VALUE;
 import static org.apache.hadoop.hbase.regionserver.HStoreFile.TIMERANGE_KEY;
 import java.io.IOException;
 import java.net.InetSocketAddress;
@@ -47,6 +49,7 @@ import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.io.hfile.HFileContext;
+import org.apache.hadoop.hbase.io.hfile.HFileWriterImpl;
 import org.apache.hadoop.hbase.util.BloomContext;
 import org.apache.hadoop.hbase.util.BloomFilterFactory;
 import org.apache.hadoop.hbase.util.BloomFilterUtil;
@@ -165,6 +168,9 @@ public class StoreFileWriter implements CellSink, ShipperListener {
     }
   }
 
+  public long getPos() throws IOException {
+    return ((HFileWriterImpl) writer).getPos();
+  }
   /**
    * Writes meta data.
    * Call before {@link #close()} since its written as meta data to this file.
@@ -239,6 +245,29 @@ public class StoreFileWriter implements CellSink, ShipperListener {
   }
 
   /**
+   * Appends MOB - specific metadata (even if it is empty)
+   * @param mobRefSet - set of MOB file names
+   * @throws IOException problem writing to FS
+   */
+  public void appendMobMetadata(Set<String> mobRefSet) throws IOException {
+    if (mobRefSet != null && mobRefSet.size() > 0) {
+      StringBuilder sb = new StringBuilder(2 * mobRefSet.size() - 1);
+      String[] arr = new String[mobRefSet.size()];
+      arr = mobRefSet.toArray(arr);
+      for (int i = 0; i < arr.length; i++) {
+        sb.append(arr[i]);
+        if (i < arr.length - 1) {
+          sb.append(",");
+        }
+      }
+      byte[] bytes = Bytes.toBytes(sb.toString());
+      writer.appendFileInfo(MOB_FILE_REFS, bytes);
+    } else {
+      writer.appendFileInfo(MOB_FILE_REFS, NULL_VALUE);
+    }
+  }
+
+  /**
    * Add TimestampRange and earliest put timestamp to Metadata
    */
   public void appendTrackedTimestampsToMetadata() throws IOException {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
index 10fac55..46e7a2e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
@@ -115,7 +115,7 @@ public abstract class Compactor<T extends CellSink> {
   }
 
   /** The sole reason this class exists is that java has no ref/out/pointer parameters. */
-  protected static class FileDetails {
+  public static class FileDetails {
     /** Maximum key count after compaction (for blooms) */
     public long maxKeyCount = 0;
     /** Earliest put timestamp if major compaction */
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
index 81e6fd2..00e38bb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
@@ -1140,7 +1140,7 @@ public abstract class FSUtils extends CommonFSUtils {
 
     @Override
     protected boolean accept(Path p, @CheckForNull Boolean isDir) {
-      if (!StoreFileInfo.isHFile(p)) {
+      if (!StoreFileInfo.isHFile(p) && !StoreFileInfo.isMobFile(p)) {
         return false;
       }
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/FaultyMobStoreCompactor.java
similarity index 50%
copy from hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java
copy to hbase-server/src/test/java/org/apache/hadoop/hbase/mob/FaultyMobStoreCompactor.java
index e73a7d2..057d81f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/FaultyMobStoreCompactor.java
@@ -1,4 +1,5 @@
 /**
+ *
  * 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
@@ -17,158 +18,103 @@
  */
 package org.apache.hadoop.hbase.mob;
 
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.util.ArrayList;
 import java.util.Date;
 import java.util.List;
+import java.util.Random;
+import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.PrivateCellUtil;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.PrivateCellUtil;
+import org.apache.hadoop.hbase.io.hfile.CorruptHFileException;
 import org.apache.hadoop.hbase.regionserver.CellSink;
-import org.apache.hadoop.hbase.regionserver.HMobStore;
 import org.apache.hadoop.hbase.regionserver.HStore;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
 import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
-import org.apache.hadoop.hbase.regionserver.ScanInfo;
-import org.apache.hadoop.hbase.regionserver.ScanType;
 import org.apache.hadoop.hbase.regionserver.ScannerContext;
 import org.apache.hadoop.hbase.regionserver.ShipperListener;
-import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
 import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
-import org.apache.hadoop.hbase.regionserver.StoreScanner;
-import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequestImpl;
-import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactor;
 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputControlUtil;
 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
-import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
 /**
- * Compact passed set of files in the mob-enabled column family.
+ * This class is used for testing only. The main purpose is to emulate
+ * random failures during MOB compaction process.
+ * Example of usage:
+ * <pre>{@code
+ * public class SomeTest {
+ *
+ *   public void initConfiguration(Configuration conf){
+ *     conf.set(MobStoreEngine.DEFAULT_MOB_COMPACTOR_CLASS_KEY,
+         FaultyMobStoreCompactor.class.getName());
+       conf.setDouble("injected.fault.probability", 0.1);
+ *   }
+ * }
+ * }</pre>
+ * @see org.apache.hadoop.hbase.mob.TestMobCompaction on how to use and configure
+ * this class.
+ *
  */
 @InterfaceAudience.Private
-public class DefaultMobStoreCompactor extends DefaultCompactor {
+public class FaultyMobStoreCompactor extends DefaultMobStoreCompactor {
 
-  private static final Logger LOG = LoggerFactory.getLogger(DefaultMobStoreCompactor.class);
-  private long mobSizeThreshold;
-  private HMobStore mobStore;
+  private static final Logger LOG = LoggerFactory.getLogger(FaultyMobStoreCompactor.class);
 
-  private final InternalScannerFactory scannerFactory = new InternalScannerFactory() {
+  public static AtomicLong mobCounter = new AtomicLong();
+  public static AtomicLong totalFailures = new AtomicLong();
+  public static AtomicLong totalCompactions = new AtomicLong();
+  public static AtomicLong totalMajorCompactions = new AtomicLong();
 
-    @Override
-    public ScanType getScanType(CompactionRequestImpl request) {
-      // retain the delete markers until they are expired.
-      return ScanType.COMPACT_RETAIN_DELETES;
-    }
+  static double failureProb = 0.1d;
+  static Random rnd = new Random();
 
-    @Override
-    public InternalScanner createScanner(ScanInfo scanInfo, List<StoreFileScanner> scanners,
-        ScanType scanType, FileDetails fd, long smallestReadPoint) throws IOException {
-      return new StoreScanner(store, scanInfo, scanners, scanType, smallestReadPoint,
-          fd.earliestPutTs);
-    }
-  };
-
-  private final CellSinkFactory<StoreFileWriter> writerFactory =
-      new CellSinkFactory<StoreFileWriter>() {
-        @Override
-        public StoreFileWriter createWriter(InternalScanner scanner,
-            org.apache.hadoop.hbase.regionserver.compactions.Compactor.FileDetails fd,
-            boolean shouldDropBehind) throws IOException {
-          // make this writer with tags always because of possible new cells with tags.
-          return store.createWriterInTmp(fd.maxKeyCount, compactionCompression, true, true, true,
-            shouldDropBehind);
-        }
-      };
-
-  public DefaultMobStoreCompactor(Configuration conf, HStore store) {
+  public FaultyMobStoreCompactor(Configuration conf, HStore store) {
     super(conf, store);
-    // The mob cells reside in the mob-enabled column family which is held by HMobStore.
-    // During the compaction, the compactor reads the cells from the mob files and
-    // probably creates new mob files. All of these operations are included in HMobStore,
-    // so we need to cast the Store to HMobStore.
-    if (!(store instanceof HMobStore)) {
-      throw new IllegalArgumentException("The store " + store + " is not a HMobStore");
-    }
-    mobStore = (HMobStore) store;
-    mobSizeThreshold = store.getColumnFamilyDescriptor().getMobThreshold();
-  }
-
-  @Override
-  public List<Path> compact(CompactionRequestImpl request, ThroughputController throughputController,
-      User user) throws IOException {
-    return compact(request, scannerFactory, writerFactory, throughputController, user);
+    failureProb = conf.getDouble("injected.fault.probability", 0.1);
   }
 
-  /**
-   * Performs compaction on a column family with the mob flag enabled.
-   * This is for when the mob threshold size has changed or if the mob
-   * column family mode has been toggled via an alter table statement.
-   * Compacts the files by the following rules.
-   * 1. If the Put cell has a mob reference tag, the cell's value is the path of the mob file.
-   * <ol>
-   * <li>
-   * If the value size of a cell is larger than the threshold, this cell is regarded as a mob,
-   * directly copy the (with mob tag) cell into the new store file.
-   * </li>
-   * <li>
-   * Otherwise, retrieve the mob cell from the mob file, and writes a copy of the cell into
-   * the new store file.
-   * </li>
-   * </ol>
-   * 2. If the Put cell doesn't have a reference tag.
-   * <ol>
-   * <li>
-   * If the value size of a cell is larger than the threshold, this cell is regarded as a mob,
-   * write this cell to a mob file, and write the path of this mob file to the store file.
-   * </li>
-   * <li>
-   * Otherwise, directly write this cell into the store file.
-   * </li>
-   * </ol>
-   * 3. Decide how to write a Delete cell.
-   * <ol>
-   * <li>
-   * If a Delete cell does not have a mob reference tag which means this delete marker have not
-   * been written to the mob del file, write this cell to the mob del file, and write this cell
-   * with a ref tag to a store file.
-   * </li>
-   * <li>
-   * Otherwise, directly write it to a store file.
-   * </li>
-   * </ol>
-   * After the major compaction on the normal hfiles, we have a guarantee that we have purged all
-   * deleted or old version mob refs, and the delete markers are written to a del file with the
-   * suffix _del. Because of this, it is safe to use the del file in the mob compaction.
-   * The mob compaction doesn't take place in the normal hfiles, it occurs directly in the
-   * mob files. When the small mob files are merged into bigger ones, the del file is added into
-   * the scanner to filter the deleted cells.
-   * @param fd File details
-   * @param scanner Where to read from.
-   * @param writer Where to write to.
-   * @param smallestReadPoint Smallest read point.
-   * @param cleanSeqId When true, remove seqId(used to be mvcc) value which is <= smallestReadPoint
-   * @param throughputController The compaction throughput controller.
-   * @param major Is a major compaction.
-   * @param numofFilesToCompact the number of files to compact
-   * @return Whether compaction ended; false if it was interrupted for any reason.
-   */
   @Override
   protected boolean performCompaction(FileDetails fd, InternalScanner scanner, CellSink writer,
       long smallestReadPoint, boolean cleanSeqId, ThroughputController throughputController,
       boolean major, int numofFilesToCompact) throws IOException {
+
+    totalCompactions.incrementAndGet();
+    if (major) {
+      totalMajorCompactions.incrementAndGet();
+    }
     long bytesWrittenProgressForCloseCheck = 0;
     long bytesWrittenProgressForLog = 0;
     long bytesWrittenProgressForShippedCall = 0;
+    // Clear old mob references
+    mobRefSet.get().clear();
+    boolean isUserRequest = userRequest.get();
+    boolean compactMOBs = major && isUserRequest;
+    boolean discardMobMiss =
+        conf.getBoolean(MobConstants.MOB_UNSAFE_DISCARD_MISS_KEY, MobConstants.DEFAULT_MOB_DISCARD_MISS);
+
+    boolean mustFail = false;
+    if (compactMOBs) {
+      mobCounter.incrementAndGet();
+      double dv = rnd.nextDouble();
+      if (dv < failureProb) {
+        mustFail = true;
+        totalFailures.incrementAndGet();
+      }
+    }
+
+    FileSystem fs = FileSystem.get(conf);
+
     // Since scanner.next() can return 'false' but still be delivering data,
     // we have to use a do/while loop.
     List<Cell> cells = new ArrayList<>();
@@ -183,16 +129,27 @@ public class DefaultMobStoreCompactor extends DefaultCompactor {
     boolean hasMore;
     Path path = MobUtils.getMobFamilyPath(conf, store.getTableName(), store.getColumnFamilyName());
     byte[] fileName = null;
-    StoreFileWriter mobFileWriter = null, delFileWriter = null;
-    long mobCells = 0, deleteMarkersCount = 0;
+    StoreFileWriter mobFileWriter = null;
+    long mobCells = 0;
     long cellsCountCompactedToMob = 0, cellsCountCompactedFromMob = 0;
     long cellsSizeCompactedToMob = 0, cellsSizeCompactedFromMob = 0;
     boolean finished = false;
+
     ScannerContext scannerContext =
         ScannerContext.newBuilder().setBatchLimit(compactionKVMax).build();
     throughputController.start(compactionName);
-    KeyValueScanner kvs = (scanner instanceof KeyValueScanner)? (KeyValueScanner)scanner : null;
-    long shippedCallSizeLimit = (long) numofFilesToCompact * this.store.getColumnFamilyDescriptor().getBlocksize();
+    KeyValueScanner kvs = (scanner instanceof KeyValueScanner) ? (KeyValueScanner) scanner : null;
+    long shippedCallSizeLimit =
+        (long) numofFilesToCompact * this.store.getColumnFamilyDescriptor().getBlocksize();
+
+    Cell mobCell = null;
+
+    long counter = 0;
+    long countFailAt = -1;
+    if (mustFail) {
+      countFailAt = rnd.nextInt(100); // randomly fail fast
+    }
+
     try {
       try {
         // If the mob file writer could not be created, directly write the cell to the store file.
@@ -200,19 +157,15 @@ public class DefaultMobStoreCompactor extends DefaultCompactor {
           compactionCompression, store.getRegionInfo().getStartKey(), true);
         fileName = Bytes.toBytes(mobFileWriter.getPath().getName());
       } catch (IOException e) {
-        LOG.warn("Failed to create mob writer, "
-               + "we will continue the compaction by writing MOB cells directly in store files", e);
+        // Bailing out
+        LOG.error("Failed to create mob writer, ", e);
+        throw e;
       }
-      if (major) {
-        try {
-          delFileWriter = mobStore.createDelFileWriterInTmp(new Date(fd.latestPutTs),
-            fd.maxKeyCount, compactionCompression, store.getRegionInfo().getStartKey());
-        } catch (IOException e) {
-          LOG.warn(
-            "Failed to create del writer, "
-            + "we will continue the compaction by writing delete markers directly in store files",
-            e);
-        }
+      if (compactMOBs) {
+        // Add the only reference we get for compact MOB case
+        // because new store file will have only one MOB reference
+        // in this case - of newly compacted MOB file
+        mobRefSet.get().add(mobFileWriter.getPath().getName());
       }
       do {
         hasMore = scanner.next(cells, scannerContext);
@@ -220,54 +173,103 @@ public class DefaultMobStoreCompactor extends DefaultCompactor {
           now = EnvironmentEdgeManager.currentTime();
         }
         for (Cell c : cells) {
-          if (major && CellUtil.isDelete(c)) {
-            if (MobUtils.isMobReferenceCell(c) || delFileWriter == null) {
-              // Directly write it to a store file
-              writer.append(c);
+          counter++;
+          if (compactMOBs) {
+            if (MobUtils.isMobReferenceCell(c)) {
+              if (counter == countFailAt) {
+                LOG.warn("\n\n INJECTED FAULT mobCounter=" + mobCounter.get() + "\n\n");
+                throw new CorruptHFileException("injected fault");
+              }
+              String fName = MobUtils.getMobFileName(c);
+              Path pp = new Path(new Path(fs.getUri()), new Path(path, fName));
+
+              // Added to support migration
+              try {
+                mobCell = mobStore.resolve(c, true, false).getCell();
+              } catch (FileNotFoundException fnfe) {
+                if (discardMobMiss) {
+                  LOG.error("Missing MOB cell: file=" + pp + " not found");
+                  continue;
+                } else {
+                  throw fnfe;
+                }
+              }
+
+              if (discardMobMiss && mobCell.getValueLength() == 0) {
+                LOG.error("Missing MOB cell value: file=" + pp + " cell=" + mobCell);
+                continue;
+              }
+
+              if (mobCell.getValueLength() > mobSizeThreshold) {
+                // put the mob data back to the store file
+                PrivateCellUtil.setSequenceId(mobCell, c.getSequenceId());
+                mobFileWriter.append(mobCell);
+                writer.append(
+                  MobUtils.createMobRefCell(mobCell, fileName, this.mobStore.getRefCellTags()));
+                cellsCountCompactedFromMob++;
+                cellsSizeCompactedFromMob += mobCell.getValueLength();
+                mobCells++;
+              } else {
+                // If MOB value is less than threshold, append it directly to a store file
+                PrivateCellUtil.setSequenceId(mobCell, c.getSequenceId());
+                writer.append(mobCell);
+              }
+
             } else {
-              // Add a ref tag to this cell and write it to a store file.
-              writer.append(MobUtils.createMobRefDeleteMarker(c));
-              // Write the cell to a del file
-              delFileWriter.append(c);
-              deleteMarkersCount++;
+              // Not a MOB reference cell
+              int size = c.getValueLength();
+              if (size > mobSizeThreshold) {
+                mobFileWriter.append(c);
+                writer
+                    .append(MobUtils.createMobRefCell(c, fileName, this.mobStore.getRefCellTags()));
+                mobCells++;
+              } else {
+                writer.append(c);
+              }
             }
-          } else if (mobFileWriter == null || c.getTypeByte() != KeyValue.Type.Put.getCode()) {
-            // If the mob file writer is null or the kv type is not put, directly write the cell
+          } else if (c.getTypeByte() != KeyValue.Type.Put.getCode()) {
+            // Not a major compaction or major with MOB disabled
+            // If the kv type is not put, directly write the cell
             // to the store file.
             writer.append(c);
           } else if (MobUtils.isMobReferenceCell(c)) {
+            // Not a major MOB compaction, Put MOB reference
             if (MobUtils.hasValidMobRefCellValue(c)) {
               int size = MobUtils.getMobValueLength(c);
               if (size > mobSizeThreshold) {
                 // If the value size is larger than the threshold, it's regarded as a mob. Since
                 // its value is already in the mob file, directly write this cell to the store file
                 writer.append(c);
+                // Add MOB reference to a set
+                mobRefSet.get().add(MobUtils.getMobFileName(c));
               } else {
                 // If the value is not larger than the threshold, it's not regarded a mob. Retrieve
-                // the mob cell from the mob file, and write it back to the store file. Must
-                // close the mob scanner once the life cycle finished.
-                try (MobCell mobCell = mobStore.resolve(c, false)) {
-                  if (mobCell.getCell().getValueLength() != 0) {
-                    // put the mob data back to the store file
-                    PrivateCellUtil.setSequenceId(mobCell.getCell(), c.getSequenceId());
-                    writer.append(mobCell.getCell());
-                    cellsCountCompactedFromMob++;
-                    cellsSizeCompactedFromMob += mobCell.getCell().getValueLength();
-                  } else {
-                    // If the value of a file is empty, there might be issues when retrieving,
-                    // directly write the cell to the store file, and leave it to be handled by the
-                    // next compaction.
-                    writer.append(c);
-                  }
+                // the mob cell from the mob file, and write it back to the store file.
+                mobCell = mobStore.resolve(c, true, false).getCell();
+                if (mobCell.getValueLength() != 0) {
+                  // put the mob data back to the store file
+                  PrivateCellUtil.setSequenceId(mobCell, c.getSequenceId());
+                  writer.append(mobCell);
+                  cellsCountCompactedFromMob++;
+                  cellsSizeCompactedFromMob += mobCell.getValueLength();
+                } else {
+                  // If the value of a file is empty, there might be issues when retrieving,
+                  // directly write the cell to the store file, and leave it to be handled by the
+                  // next compaction.
+                  LOG.error("Empty value for: " + c);
+                  writer.append(c);
+                  // Add MOB reference to a set
+                  mobRefSet.get().add(MobUtils.getMobFileName(c));
                 }
               }
             } else {
-              LOG.warn("The value format of the KeyValue " + c
-                  + " is wrong, its length is less than " + Bytes.SIZEOF_INT);
+              // TODO ????
+              LOG.error("Corrupted MOB reference: " + c);
               writer.append(c);
             }
           } else if (c.getValueLength() <= mobSizeThreshold) {
-            //If value size of a cell is not larger than the threshold, directly write to store file
+            // If the value size of a cell is not larger than the threshold, directly write it to
+            // the store file.
             writer.append(c);
           } else {
             // If the value size of a cell is larger than the threshold, it's regarded as a mob,
@@ -275,13 +277,15 @@ public class DefaultMobStoreCompactor extends DefaultCompactor {
             mobCells++;
             // append the original keyValue in the mob file.
             mobFileWriter.append(c);
-            Cell reference = MobUtils.createMobRefCell(c, fileName,
-                this.mobStore.getRefCellTags());
+            Cell reference = MobUtils.createMobRefCell(c, fileName, this.mobStore.getRefCellTags());
             // write the cell whose value is the path of a mob file to the store file.
             writer.append(reference);
             cellsCountCompactedToMob++;
             cellsSizeCompactedToMob += c.getValueLength();
+            // Add ref we get for compact MOB case
+            mobRefSet.get().add(mobFileWriter.getPath().getName());
           }
+
           int len = c.getSerializedSize();
           ++progress.currentCompactedKVs;
           progress.totalCompactedSize += len;
@@ -302,7 +306,7 @@ public class DefaultMobStoreCompactor extends DefaultCompactor {
             }
           }
           if (kvs != null && bytesWrittenProgressForShippedCall > shippedCallSizeLimit) {
-            ((ShipperListener)writer).beforeShipped();
+            ((ShipperListener) writer).beforeShipped();
             kvs.shipped();
             bytesWrittenProgressForShippedCall = 0;
           }
@@ -326,6 +330,12 @@ public class DefaultMobStoreCompactor extends DefaultCompactor {
       progress.cancel();
       throw new InterruptedIOException(
           "Interrupted while control throughput of compacting " + compactionName);
+    } catch (FileNotFoundException e) {
+      LOG.error("MOB Stress Test FAILED, region: " + store.getRegionInfo().getEncodedName(), e);
+      System.exit(-1);
+    } catch (IOException t) {
+      LOG.error("Mob compaction failed for region: " + store.getRegionInfo().getEncodedName());
+      throw t;
     } finally {
       // Clone last cell in the final because writer will append last cell when committing. If
       // don't clone here and once the scanner get closed, then the memory of last cell will be
@@ -333,24 +343,13 @@ public class DefaultMobStoreCompactor extends DefaultCompactor {
       ((ShipperListener) writer).beforeShipped();
       throughputController.finish(compactionName);
       if (!finished && mobFileWriter != null) {
+        // Remove all MOB references because compaction failed
+        mobRefSet.get().clear();
+        // Abort writer
         abortWriter(mobFileWriter);
       }
-      if (!finished && delFileWriter != null) {
-        abortWriter(delFileWriter);
-      }
-    }
-    if (delFileWriter != null) {
-      if (deleteMarkersCount > 0) {
-        // If the del file is not empty, commit it.
-        // If the commit fails, the compaction is re-performed again.
-        delFileWriter.appendMetadata(fd.maxSeqId, major, deleteMarkersCount);
-        delFileWriter.close();
-        mobStore.commitFile(delFileWriter.getPath(), path);
-      } else {
-        // If the del file is empty, delete it instead of committing.
-        abortWriter(delFileWriter);
-      }
     }
+
     if (mobFileWriter != null) {
       if (mobCells > 0) {
         // If the mob file is not empty, commit it.
@@ -368,5 +367,7 @@ public class DefaultMobStoreCompactor extends DefaultCompactor {
     mobStore.updateCellsSizeCompactedToMob(cellsSizeCompactedToMob);
     progress.complete();
     return true;
+
   }
+
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobStressTool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobStressTool.java
new file mode 100644
index 0000000..146fb21
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobStressTool.java
@@ -0,0 +1,79 @@
+/**
+ * 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.mob;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.util.AbstractHBaseTool;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class MobStressTool extends AbstractHBaseTool {
+  private static final Logger LOG = LoggerFactory.getLogger(MobStressTool.class);
+  private CommandLine cmd;
+
+  public MobStressTool() throws IOException {
+    init();
+  }
+
+  protected void init() throws IOException {
+    // define supported options
+    addOptWithArg("n", "Number of MOB key-values to insert, default - 10000000");
+  }
+
+  @Override
+  protected void addOptions() {
+  }
+
+  @Override
+  protected void processOptions(CommandLine cmd) {
+    this.cmd = cmd;
+  }
+
+  @Override
+  protected int doWork() throws Exception {
+    long numRowsToInsert = 10000000;
+    if (cmd.hasOption("n")) {
+      numRowsToInsert = Long.parseLong(cmd.getOptionValue("n"));
+      if (numRowsToInsert < 0) {
+        LOG.warn("Ignore wrong option '-n'");
+        numRowsToInsert = 10000000;
+      }
+    }
+
+    TestMobCompaction test = new TestMobCompaction();
+    test.init(getConf(), numRowsToInsert);
+    test.testMobCompaction();
+    return 0;
+  }
+
+  public static void main(String[] args) throws Exception {
+    Configuration conf = HBaseConfiguration.create();
+    int ret = ToolRunner.run(conf, new MobStressTool(), args);
+    System.exit(ret);
+  }
+
+}
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
new file mode 100644
index 0000000..deb1cd4
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TesMobFileCleanerChore.java
@@ -0,0 +1,236 @@
+/**
+ *
+ * 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.mob;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.CompactionState;
+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;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+  * Mob file cleaner chore test.
+  * 1. Creates MOB table
+  * 2. Load MOB data and flushes it N times
+  * 3. Runs major MOB compaction (N MOB files go to archive)
+  * 4. Verifies that number of MOB files in a mob directory is N+1
+  * 5. Waits for a period of time larger than minimum age to archive
+  * 6. Runs Mob cleaner chore
+  * 7 Verifies that number of MOB files in a mob directory is 1.
+ */
+@SuppressWarnings("deprecation")
+@Category(MediumTests.class)
+public class TesMobFileCleanerChore {
+  private static final Logger LOG = LoggerFactory.getLogger(TesMobFileCleanerChore.class);
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TesMobFileCleanerChore.class);
+  @Rule
+  public TestName testName = new TestName();
+
+  private HBaseTestingUtility HTU;
+
+  private final static String famStr = "f1";
+  private final static byte[] fam = Bytes.toBytes(famStr);
+  private final static byte[] qualifier = Bytes.toBytes("q1");
+  private final static long mobLen = 10;
+  private final static byte[] mobVal = Bytes
+      .toBytes("01234567890123456789012345678901234567890123456789012345678901234567890123456789");
+
+  private Configuration conf;
+  private HTableDescriptor hdt;
+  private HColumnDescriptor hcd;
+  private Admin admin;
+  private Table table = null;
+  private MobFileCleanerChore chore;
+  private long minAgeToArchive = 10000;
+
+  public TesMobFileCleanerChore() {
+  }
+
+
+  @Before
+  public void setUp() throws Exception {
+    HTU = new HBaseTestingUtility();
+    hdt = HTU.createTableDescriptor("testMobCompactTable");
+    conf = HTU.getConfiguration();
+
+    initConf();
+
+    HTU.startMiniCluster();
+    admin = HTU.getAdmin();
+    chore = new MobFileCleanerChore();
+    hcd = new HColumnDescriptor(fam);
+    hcd.setMobEnabled(true);
+    hcd.setMobThreshold(mobLen);
+    hcd.setMaxVersions(1);
+    hdt.addFamily(hcd);
+    table = HTU.createTable(hdt, null);
+  }
+
+  private void initConf() {
+
+    conf.setInt("hfile.format.version", 3);
+    conf.setLong(TimeToLiveHFileCleaner.TTL_CONF_KEY, 0);
+    conf.setInt("hbase.client.retries.number", 100);
+    conf.setInt("hbase.hregion.max.filesize", 200000000);
+    conf.setInt("hbase.hregion.memstore.flush.size", 800000);
+    conf.setInt("hbase.hstore.blockingStoreFiles", 150);
+    conf.setInt("hbase.hstore.compaction.throughput.lower.bound", 52428800);
+    conf.setInt("hbase.hstore.compaction.throughput.higher.bound", 2 * 52428800);
+    //conf.set(MobStoreEngine.DEFAULT_MOB_COMPACTOR_CLASS_KEY,
+    //  FaultyMobStoreCompactor.class.getName());
+    // Disable automatic MOB compaction
+    conf.setLong(MobConstants.MOB_COMPACTION_CHORE_PERIOD, 0);
+    // Disable automatic MOB file cleaner chore
+    conf.setLong(MobConstants.MOB_CLEANER_PERIOD, 0);
+    // Set minimum age to archive to 10 sec
+    conf.setLong(MobConstants.MIN_AGE_TO_ARCHIVE_KEY, minAgeToArchive);
+    // Set compacted file discharger interval to a half minAgeToArchive
+    conf.setLong("hbase.hfile.compaction.discharger.interval", minAgeToArchive/2);
+  }
+
+  private void loadData(int start, int num) {
+    try {
+
+      for (int i = 0; i < num; i++) {
+        Put p = new Put(Bytes.toBytes(start + i));
+        p.addColumn(fam, qualifier, mobVal);
+        table.put(p);
+      }
+      admin.flush(table.getName());
+    } catch (Exception e) {
+      LOG.error("MOB file cleaner chore test FAILED", e);
+      assertTrue(false);
+    }
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    HTU.shutdownMiniCluster();
+  }
+
+  @Test
+  public void testMobFileCleanerChore() throws InterruptedException, IOException {
+
+    try {
+
+      loadData(0, 10);
+      loadData(10, 10);
+      loadData(20, 10);
+      long num = getNumberOfMobFiles(conf, table.getName(), new String(fam));
+      assertEquals(3, num);
+      // Major compact
+      admin.majorCompact(hdt.getTableName(), fam);
+      // wait until compaction is complete
+      while (admin.getCompactionState(hdt.getTableName()) != CompactionState.NONE) {
+        Thread.sleep(100);
+      }
+
+      num = getNumberOfMobFiles(conf, table.getName(), new String(fam));
+      assertEquals(4, num);
+      // We have guarantee, that compcated file discharger will run during this pause
+      // because it has interval less than this wait time
+      LOG.info("Waiting for {}ms", minAgeToArchive + 1000);
+
+      Thread.sleep(minAgeToArchive + 1000);
+      LOG.info("Cleaning up MOB files");
+      // Cleanup again
+      chore.cleanupObsoleteMobFiles(conf, table.getName());
+
+      num = getNumberOfMobFiles(conf, table.getName(), new String(fam));
+      assertEquals(1, num);
+
+      long scanned = scanTable();
+      assertEquals(30, scanned);
+
+    } finally {
+
+      admin.disableTable(hdt.getTableName());
+      admin.deleteTable(hdt.getTableName());
+    }
+    LOG.info("MOB Stress Test finished OK");
+
+  }
+
+  private  long getNumberOfMobFiles(Configuration conf, TableName tableName, String family)
+      throws IOException {
+    FileSystem fs = FileSystem.get(conf);
+    Path dir = MobUtils.getMobFamilyPath(conf, tableName, family);
+    FileStatus[] stat = fs.listStatus(dir);
+    for (FileStatus st : stat) {
+      LOG.debug("MOB Directory content: {}", st.getPath());
+    }
+    LOG.debug("MOB Directory content total files: {}", stat.length);
+
+    return stat.length;
+  }
+
+
+  private long scanTable() {
+    try {
+
+      Result result;
+      ResultScanner scanner = table.getScanner(fam);
+      long counter = 0;
+      while ((result = scanner.next()) != null) {
+        assertTrue(Arrays.equals(result.getValue(fam, qualifier), mobVal));
+        counter++;
+      }
+      return counter;
+    } catch (Exception e) {
+      e.printStackTrace();
+      LOG.error("MOB file cleaner chore test FAILED");
+      if (HTU != null) {
+        assertTrue(false);
+      } else {
+        System.exit(-1);
+      }
+    }
+    return 0;
+  }
+}
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
new file mode 100644
index 0000000..2b2cac3
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompaction.java
@@ -0,0 +1,375 @@
+/**
+ *
+ * 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.mob;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.KeepDeletedCells;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.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;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Ignore;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+/**
+    Reproduction for MOB data loss
+
+ 1. Settings: Region Size 200 MB,  Flush threshold 800 KB.
+ 2. Insert 10 Million records
+ 3. MOB Compaction and Archiver
+      a) Trigger MOB Compaction (every 2 minutes)
+      b) Trigger major compaction (every 2 minutes)
+      c) Trigger archive cleaner (every 3 minutes)
+ 4. Validate MOB data after complete data load.
+
+ */
+@SuppressWarnings("deprecation")
+@Category(LargeTests.class)
+public class TestMobCompaction {
+  private static final Logger LOG = LoggerFactory.getLogger(TestMobCompaction.class);
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestMobCompaction.class);
+  @Rule
+  public TestName testName = new TestName();
+
+  private HBaseTestingUtility HTU;
+
+  private final static String famStr = "f1";
+  private final static byte[] fam = Bytes.toBytes(famStr);
+  private final static byte[] qualifier = Bytes.toBytes("q1");
+  private final static long mobLen = 10;
+  private final static byte[] mobVal = Bytes
+      .toBytes("01234567890123456789012345678901234567890123456789012345678901234567890123456789");
+
+  private Configuration conf;
+  private HTableDescriptor hdt;
+  private HColumnDescriptor hcd;
+  private Admin admin;
+  private long count = 500000;
+  private double failureProb = 0.1;
+  private Table table = null;
+  private MobFileCleanerChore chore = new MobFileCleanerChore();
+
+  private static volatile boolean run = true;
+
+  public TestMobCompaction() {
+
+  }
+
+  public void init(Configuration conf, long numRows) throws IOException {
+    this.conf = conf;
+    this.count = numRows;
+    printConf();
+    hdt = createTableDescriptor("testMobCompactTable");
+    Connection conn = ConnectionFactory.createConnection(this.conf);
+    this.admin = conn.getAdmin();
+    this.hcd = new HColumnDescriptor(fam);
+    this.hcd.setMobEnabled(true);
+    this.hcd.setMobThreshold(mobLen);
+    this.hcd.setMaxVersions(1);
+    this.hdt.addFamily(hcd);
+    if (admin.tableExists(hdt.getTableName())) {
+      admin.disableTable(hdt.getTableName());
+      admin.deleteTable(hdt.getTableName());
+    }
+    admin.createTable(hdt);
+    table = conn.getTable(hdt.getTableName());
+  }
+
+  private void printConf() {
+    LOG.info("To run stress test, please change HBase configuration as following:");
+    LOG.info("hfile.format.version=3");
+    LOG.info("hbase.master.hfilecleaner.ttl=0");
+    LOG.info("hbase.hregion.max.filesize=200000000");
+    LOG.info("hbase.client.retries.number=100");
+    LOG.info("hbase.hregion.memstore.flush.size=800000");
+    LOG.info("hbase.hstore.blockingStoreFiles=150");
+    LOG.info("hbase.hstore.compaction.throughput.lower.bound=50000000");
+    LOG.info("hbase.hstore.compaction.throughput.higher.bound=100000000");
+    LOG.info("hbase.master.mob.cleaner.period=0");
+    LOG.info("hbase.mob.default.compactor=org.apache.hadoop.hbase.mob.FaultyMobStoreCompactor");
+    LOG.warn("injected.fault.probability=x, where x is between 0. and 1.");
+
+  }
+
+  private HTableDescriptor createTableDescriptor(final String name, final int minVersions,
+      final int versions, final int ttl, KeepDeletedCells keepDeleted) {
+    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name));
+    return htd;
+  }
+
+  private HTableDescriptor createTableDescriptor(final String name) {
+    return createTableDescriptor(name, HColumnDescriptor.DEFAULT_MIN_VERSIONS, 1,
+      HConstants.FOREVER, HColumnDescriptor.DEFAULT_KEEP_DELETED);
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    HTU = new HBaseTestingUtility();
+    hdt = HTU.createTableDescriptor("testMobCompactTable");
+    conf = HTU.getConfiguration();
+
+    initConf();
+
+    // HTU.getConfiguration().setInt("hbase.mob.compaction.chore.period", 0);
+    HTU.startMiniCluster();
+    admin = HTU.getAdmin();
+
+    hcd = new HColumnDescriptor(fam);
+    hcd.setMobEnabled(true);
+    hcd.setMobThreshold(mobLen);
+    hcd.setMaxVersions(1);
+    hdt.addFamily(hcd);
+    table = HTU.createTable(hdt, null);
+  }
+
+  private void initConf() {
+
+    conf.setInt("hfile.format.version", 3);
+    conf.setLong(TimeToLiveHFileCleaner.TTL_CONF_KEY, 0);
+    conf.setInt("hbase.client.retries.number", 100);
+    conf.setInt("hbase.hregion.max.filesize", 200000000);
+    conf.setInt("hbase.hregion.memstore.flush.size", 800000);
+    conf.setInt("hbase.hstore.blockingStoreFiles", 150);
+    conf.setInt("hbase.hstore.compaction.throughput.lower.bound", 52428800);
+    conf.setInt("hbase.hstore.compaction.throughput.higher.bound", 2 * 52428800);
+    conf.setDouble("injected.fault.probability", failureProb);
+//    conf.set(MobStoreEngine.DEFAULT_MOB_COMPACTOR_CLASS_KEY,
+//      FaultyMobStoreCompactor.class.getName());
+    conf.setLong(MobConstants.MOB_COMPACTION_CHORE_PERIOD, 0);
+    conf.setLong(MobConstants.MOB_CLEANER_PERIOD, 0);
+    conf.setLong(MobConstants.MIN_AGE_TO_ARCHIVE_KEY, 120000);
+    conf.set(MobConstants.MOB_COMPACTION_TYPE_KEY,
+      MobConstants.IO_OPTIMIZED_MOB_COMPACTION_TYPE);
+    conf.setLong(MobConstants.MOB_COMPACTION_MAX_FILE_SIZE_KEY, 1000000);
+
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    HTU.shutdownMiniCluster();
+  }
+
+  class MajorCompaction implements Runnable {
+
+    @Override
+    public void run() {
+      while (run) {
+        try {
+          admin.majorCompact(hdt.getTableName(), fam);
+          Thread.sleep(120000);
+        } catch (Exception e) {
+          LOG.error("MOB Stress Test FAILED", e);
+          System.exit(-1);
+        }
+      }
+    }
+  }
+
+  class CleanMobAndArchive implements Runnable {
+
+    @Override
+    public void run() {
+      while (run) {
+        try {
+          LOG.info("MOB cleanup chore started ...");
+          chore.cleanupObsoleteMobFiles(conf, table.getName());
+          LOG.info("MOB cleanup chore finished");
+
+          Thread.sleep(130000);
+        } catch (Exception e) {
+          e.printStackTrace();
+        }
+      }
+    }
+  }
+
+  class WriteData implements Runnable {
+
+    private long rows = -1;
+
+    public WriteData(long rows) {
+      this.rows = rows;
+    }
+
+    @Override
+    public void run() {
+      try {
+
+        // Put Operation
+        for (int i = 0; i < rows; i++) {
+          byte[] key = Bytes.toBytes(i);
+          Put p = new Put(key);
+          p.addColumn(fam, qualifier, Bytes.add(key,mobVal));
+          table.put(p);
+          if (i % 10000 == 0) {
+            LOG.info("LOADED=" + i);
+            try {
+              Thread.sleep(500);
+            } catch (InterruptedException ee) {
+            }
+          }
+          if (i % 100000 == 0) {
+            printStats(i);
+          }
+        }
+        admin.flush(table.getName());
+        run = false;
+      } catch (Exception e) {
+        LOG.error("MOB Stress Test FAILED", e);
+        System.exit(-1);
+      }
+    }
+  }
+
+  @Ignore
+  @Test
+  public void testMobCompaction() throws InterruptedException, IOException {
+
+    try {
+
+      Thread writeData = new Thread(new WriteData(count));
+      writeData.start();
+
+      Thread majorcompact = new Thread(new MajorCompaction());
+      majorcompact.start();
+
+      Thread cleaner = new Thread(new CleanMobAndArchive());
+      cleaner.start();
+
+      while (run) {
+        Thread.sleep(1000);
+      }
+
+      getNumberOfMobFiles(conf, table.getName(), new String(fam));
+      LOG.info("Waiting for write thread to finish ...");
+      writeData.join();
+      // Cleanup again
+      chore.cleanupObsoleteMobFiles(conf, table.getName());
+      getNumberOfMobFiles(conf, table.getName(), new String(fam));
+
+      if (HTU != null) {
+        LOG.info("Archive cleaner started ...");
+        // Call archive cleaner again
+        HTU.getMiniHBaseCluster().getMaster().getHFileCleaner().choreForTesting();
+        LOG.info("Archive cleaner finished");
+      }
+
+      scanTable();
+
+    } finally {
+
+      admin.disableTable(hdt.getTableName());
+      admin.deleteTable(hdt.getTableName());
+    }
+    LOG.info("MOB Stress Test finished OK");
+    printStats(count);
+
+  }
+
+  private  long getNumberOfMobFiles(Configuration conf, TableName tableName, String family)
+      throws IOException {
+    FileSystem fs = FileSystem.get(conf);
+    Path dir = MobUtils.getMobFamilyPath(conf, tableName, family);
+    FileStatus[] stat = fs.listStatus(dir);
+    long size = 0;
+    for (FileStatus st : stat) {
+      LOG.debug("MOB Directory content: {} len={}", st.getPath(), st.getLen());
+      size+= st.getLen();
+    }
+    LOG.debug("MOB Directory content total files: {}, total size={}", stat.length, size);
+
+    return stat.length;
+  }
+
+  public void printStats(long loaded) {
+    LOG.info("MOB Stress Test: loaded=" + loaded + " compactions="
+        + FaultyMobStoreCompactor.totalCompactions.get() + " major="
+        + FaultyMobStoreCompactor.totalMajorCompactions.get() + " mob="
+        + FaultyMobStoreCompactor.mobCounter.get() + " injected failures="
+        + FaultyMobStoreCompactor.totalFailures.get());
+  }
+
+  private void scanTable() {
+    try {
+
+      Result result;
+      ResultScanner scanner = table.getScanner(fam);
+      int counter = 0;
+      while ((result = scanner.next()) != null) {
+        byte[] key = result.getRow();
+        assertTrue(Arrays.equals(result.getValue(fam, qualifier),
+          Bytes.add(key,mobVal)));
+        if (counter % 10000 == 0) {
+          LOG.info("GET=" + counter+" key=" + Bytes.toInt(key));
+        }
+        counter++;
+      }
+
+//      for (int i=0; i < count; i++) {
+//        byte[] key = Bytes.toBytes(i);
+//        Get get = new Get(key);
+//        Result res = table.get(get);
+//        assertTrue(Arrays.equals(res.getValue(fam, qualifier),
+//          Bytes.add(key,mobVal)));
+//        if (i % 1000 == 0) {
+//          LOG.info("GET=" + i);
+//        }
+//      }
+      assertEquals(count, counter);
+    } catch (Exception e) {
+      e.printStackTrace();
+      LOG.error("MOB Stress Test FAILED");
+      if (HTU != null) {
+        assertTrue(false);
+      } else {
+        System.exit(-1);
+      }
+    }
+  }
+}
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
new file mode 100644
index 0000000..169c74c
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionBase.java
@@ -0,0 +1,242 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.mob;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Random;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.CompactionState;
+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;
+import org.junit.Before;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+  * Mob file compaction base test.
+  * 1. Enables batch mode for regular MOB compaction,
+  *    Sets batch size to 7 regions. (Optional)
+  * 2. Disables periodic MOB compactions, sets minimum age to archive to 10 sec
+  * 3. Creates MOB table with 20 regions
+  * 4. Loads MOB data (randomized keys, 1000 rows), flushes data.
+  * 5. Repeats 4. two more times
+  * 6. Verifies that we have 20 *3 = 60 mob files (equals to number of regions x 3)
+  * 7. Runs major MOB compaction.
+  * 8. Verifies that number of MOB files in a mob directory is 20 x4 = 80
+  * 9. Waits for a period of time larger than minimum age to archive
+  * 10. Runs Mob cleaner chore
+  * 11 Verifies that number of MOB files in a mob directory is 20.
+  * 12 Runs scanner and checks all 3 * 1000 rows.
+ */
+@SuppressWarnings("deprecation")
+public abstract class TestMobCompactionBase {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestMobCompactionBase.class);
+
+  protected HBaseTestingUtility HTU;
+
+  protected final static String famStr = "f1";
+  protected final static byte[] fam = Bytes.toBytes(famStr);
+  protected final static byte[] qualifier = Bytes.toBytes("q1");
+  protected final static long mobLen = 10;
+  protected final static byte[] mobVal = Bytes
+      .toBytes("01234567890123456789012345678901234567890123456789012345678901234567890123456789");
+
+  protected Configuration conf;
+  protected HTableDescriptor hdt;
+  private HColumnDescriptor hcd;
+  protected Admin admin;
+  protected Table table = null;
+  protected long minAgeToArchive = 10000;
+  protected int numRegions = 20;
+  protected int rows = 1000;
+
+  protected MobFileCleanerChore cleanerChore;
+
+  public TestMobCompactionBase() {
+  }
+
+
+  @Before
+  public void setUp() throws Exception {
+    HTU = new HBaseTestingUtility();
+    hdt = HTU.createTableDescriptor(getClass().getName());
+    conf = HTU.getConfiguration();
+
+    initConf();
+
+    HTU.startMiniCluster();
+    admin = HTU.getAdmin();
+    cleanerChore = new MobFileCleanerChore();
+    hcd = new HColumnDescriptor(fam);
+    hcd.setMobEnabled(true);
+    hcd.setMobThreshold(mobLen);
+    hcd.setMaxVersions(1);
+    hdt.addFamily(hcd);
+    byte[][] splitKeys = generateSplitKeys();
+    table = HTU.createTable(hdt, splitKeys);
+
+  }
+
+  private byte[][] generateSplitKeys() {
+    RegionSplitter.UniformSplit splitAlgo = new RegionSplitter.UniformSplit();
+    return splitAlgo.split(numRegions);
+  }
+
+
+  protected void initConf() {
+
+    conf.setInt("hfile.format.version", 3);
+    // Disable automatic MOB compaction
+    conf.setLong(MobConstants.MOB_COMPACTION_CHORE_PERIOD, 0);
+    // Disable automatic MOB file cleaner chore
+    conf.setLong(MobConstants.MOB_CLEANER_PERIOD, 0);
+    // Set minimum age to archive to 10 sec
+    conf.setLong(MobConstants.MIN_AGE_TO_ARCHIVE_KEY, minAgeToArchive);
+    // Set compacted file discharger interval to a half minAgeToArchive
+    conf.setLong("hbase.hfile.compaction.discharger.interval", minAgeToArchive/2);
+  }
+
+  private void loadData(int num) {
+
+    Random r = new Random();
+    try {
+      LOG.info("Started loading {} rows", num);
+      for (int i = 0; i < num; i++) {
+        byte[] key = new byte[32];
+        r.nextBytes(key);
+        Put p = new Put(key);
+        p.addColumn(fam, qualifier, mobVal);
+        table.put(p);
+      }
+      admin.flush(table.getName());
+      LOG.info("Finished loading {} rows", num);
+    } catch (Exception e) {
+      LOG.error("MOB file compaction chore test FAILED", e);
+      assertTrue(false);
+    }
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    HTU.shutdownMiniCluster();
+  }
+
+
+  public void baseTestMobFileCompaction() throws InterruptedException, IOException {
+
+    try {
+
+      // Load and flush data 3 times
+      loadData(rows);
+      loadData(rows);
+      loadData(rows);
+      long num = getNumberOfMobFiles(conf, table.getName(), new String(fam));
+      assertEquals(numRegions * 3, num);
+      // Major MOB compact
+      mobCompact(admin, hdt, hcd);
+      // wait until compaction is complete
+      while (admin.getCompactionState(hdt.getTableName()) != CompactionState.NONE) {
+        Thread.sleep(100);
+      }
+
+      num = getNumberOfMobFiles(conf, table.getName(), new String(fam));
+      assertEquals(numRegions * 4, num);
+      // We have guarantee, that compcated file discharger will run during this pause
+      // because it has interval less than this wait time
+      LOG.info("Waiting for {}ms", minAgeToArchive + 1000);
+
+      Thread.sleep(minAgeToArchive + 1000);
+      LOG.info("Cleaning up MOB files");
+      // Cleanup again
+      cleanerChore.cleanupObsoleteMobFiles(conf, table.getName());
+
+      num = getNumberOfMobFiles(conf, table.getName(), new String(fam));
+      assertEquals(numRegions, num);
+
+      long scanned = scanTable();
+      assertEquals(3 * rows, scanned);
+
+    } finally {
+
+      admin.disableTable(hdt.getTableName());
+      admin.deleteTable(hdt.getTableName());
+    }
+
+  }
+
+  protected abstract void mobCompact(Admin admin2, HTableDescriptor hdt2, HColumnDescriptor hcd2)
+      throws IOException, InterruptedException;
+
+
+  protected  long getNumberOfMobFiles(Configuration conf, TableName tableName, String family)
+      throws IOException {
+    FileSystem fs = FileSystem.get(conf);
+    Path dir = MobUtils.getMobFamilyPath(conf, tableName, family);
+    FileStatus[] stat = fs.listStatus(dir);
+    for (FileStatus st : stat) {
+      LOG.debug("MOB Directory content: {}", st.getPath());
+    }
+    LOG.debug("MOB Directory content total files: {}", stat.length);
+
+    return stat.length;
+  }
+
+
+  protected long scanTable() {
+    try {
+
+      Result result;
+      ResultScanner scanner = table.getScanner(fam);
+      long counter = 0;
+      while ((result = scanner.next()) != null) {
+        assertTrue(Arrays.equals(result.getValue(fam, qualifier), mobVal));
+        counter++;
+      }
+      return counter;
+    } catch (Exception e) {
+      e.printStackTrace();
+      LOG.error("MOB file compaction test FAILED");
+      if (HTU != null) {
+        assertTrue(false);
+      } else {
+        System.exit(-1);
+      }
+    }
+    return 0;
+  }
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptMode.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptMode.java
new file mode 100644
index 0000000..4b7bea2
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptMode.java
@@ -0,0 +1,88 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.mob;
+import java.io.IOException;
+
+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.testclassification.LargeTests;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+  * Mob file compaction chore in a generational non-batch mode test.
+  * 1. Uses default (non-batch) mode for regular MOB compaction, sets generational mode ON
+  * 2. Disables periodic MOB compactions, sets minimum age to archive to 10 sec
+  * 3. Creates MOB table with 20 regions
+  * 4. Loads MOB data (randomized keys, 1000 rows), flushes data.
+  * 5. Repeats 4. two more times
+  * 6. Verifies that we have 20 *3 = 60 mob files (equals to number of regions x 3)
+  * 7. Runs major MOB compaction.
+  * 8. Verifies that number of MOB files in a mob directory is 20 x4 = 80
+  * 9. Waits for a period of time larger than minimum age to archive
+  * 10. Runs Mob cleaner chore
+  * 11 Verifies that number of MOB files in a mob directory is 20.
+  * 12 Runs scanner and checks all 3 * 1000 rows.
+ */
+@SuppressWarnings("deprecation")
+@Category(LargeTests.class)
+public class TestMobCompactionOptMode extends TestMobCompactionBase{
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestMobCompactionOptMode.class);
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestMobCompactionOptMode.class);
+  @Rule
+  public TestName testName = new TestName();
+
+
+  public TestMobCompactionOptMode() {
+  }
+
+  @Override
+  protected void initConf() {
+    super.initConf();
+    conf.set(MobConstants.MOB_COMPACTION_TYPE_KEY,
+      MobConstants.IO_OPTIMIZED_MOB_COMPACTION_TYPE);
+    conf.setLong(MobConstants.MOB_COMPACTION_MAX_FILE_SIZE_KEY, 1000000);
+  }
+
+  @Test
+  public void testMobFileCompactionBatchMode() throws InterruptedException, IOException {
+    LOG.info("MOB compaction generational (non-batch) mode started");
+    baseTestMobFileCompaction();
+    LOG.info("MOB compaction generational (non-batch) mode finished OK");
+
+  }
+
+  @Override
+  protected void mobCompact(Admin admin, HTableDescriptor hdt, HColumnDescriptor hcd)
+      throws IOException, InterruptedException {
+    // Major compact MOB table
+    admin.majorCompact(hdt.getTableName(), hcd.getName());
+  }
+
+}
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
new file mode 100644
index 0000000..ba5e47e
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptRegionBatchMode.java
@@ -0,0 +1,99 @@
+/**
+ *
+ * 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.mob;
+import java.io.IOException;
+
+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;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+  * Mob file compaction chore in a generational batch mode test.
+  * 1. Enables batch mode for regular MOB compaction,
+  *    Sets batch size to 7 regions. Enables generational mode.
+  * 2. Disables periodic MOB compactions, sets minimum age to archive to 10 sec
+  * 3. Creates MOB table with 20 regions
+  * 4. Loads MOB data (randomized keys, 1000 rows), flushes data.
+  * 5. Repeats 4. two more times
+  * 6. Verifies that we have 20 *3 = 60 mob files (equals to number of regions x 3)
+  * 7. Runs major MOB compaction.
+  * 8. Verifies that number of MOB files in a mob directory is 20 x4 = 80
+  * 9. Waits for a period of time larger than minimum age to archive
+  * 10. Runs Mob cleaner chore
+  * 11 Verifies that number of MOB files in a mob directory is 20.
+  * 12 Runs scanner and checks all 3 * 1000 rows.
+ */
+@SuppressWarnings("deprecation")
+@Category(LargeTests.class)
+public class TestMobCompactionOptRegionBatchMode extends TestMobCompactionBase{
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestMobCompactionOptRegionBatchMode.class);
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestMobCompactionOptRegionBatchMode.class);
+  @Rule
+  public TestName testName = new TestName();
+
+  private int batchSize = 7;
+  private MobFileCompactionChore compactionChore;
+
+  public TestMobCompactionOptRegionBatchMode() {
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    super.setUp();
+    compactionChore = new MobFileCompactionChore(conf, batchSize);
+  }
+
+  protected void initConf() {
+    super.initConf();
+    conf.setInt(MobConstants.MOB_MAJOR_COMPACTION_REGION_BATCH_SIZE, batchSize);
+    conf.set(MobConstants.MOB_COMPACTION_TYPE_KEY,
+      MobConstants.IO_OPTIMIZED_MOB_COMPACTION_TYPE);
+    conf.setLong(MobConstants.MOB_COMPACTION_MAX_FILE_SIZE_KEY, 1000000);
+  }
+
+  @Test
+  public void testMobFileCompactionBatchMode() throws InterruptedException, IOException {
+    LOG.info("MOB compaction chore generational batch mode started");
+    baseTestMobFileCompaction();
+    LOG.info("MOB compaction chore generational batch mode finished OK");
+
+  }
+
+  @Override
+  protected void mobCompact(Admin admin, HTableDescriptor hdt, HColumnDescriptor hcd)
+      throws IOException, InterruptedException {
+    // Major compact with batch mode enabled
+    compactionChore.performMajorCompactionInBatches(admin, hdt, hcd);
+  }
+
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularMode.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularMode.java
new file mode 100644
index 0000000..cec57e8
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularMode.java
@@ -0,0 +1,80 @@
+/**
+ *
+ * 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.mob;
+import java.io.IOException;
+
+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.testclassification.LargeTests;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+  * Mob file compaction chore in a regular non-batch mode test.
+  * 1. Uses default (non-batch) mode for regular MOB compaction,
+  * 2. Disables periodic MOB compactions, sets minimum age to archive to 10 sec
+  * 3. Creates MOB table with 20 regions
+  * 4. Loads MOB data (randomized keys, 1000 rows), flushes data.
+  * 5. Repeats 4. two more times
+  * 6. Verifies that we have 20 *3 = 60 mob files (equals to number of regions x 3)
+  * 7. Runs major MOB compaction.
+  * 8. Verifies that number of MOB files in a mob directory is 20 x4 = 80
+  * 9. Waits for a period of time larger than minimum age to archive
+  * 10. Runs Mob cleaner chore
+  * 11 Verifies that number of MOB files in a mob directory is 20.
+  * 12 Runs scanner and checks all 3 * 1000 rows.
+ */
+@SuppressWarnings("deprecation")
+@Category(LargeTests.class)
+public class TestMobCompactionRegularMode extends TestMobCompactionBase{
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestMobCompactionRegularMode.class);
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestMobCompactionRegularMode.class);
+  @Rule
+  public TestName testName = new TestName();
+
+
+  public TestMobCompactionRegularMode() {
+  }
+
+  @Test
+  public void testMobFileCompactionBatchMode() throws InterruptedException, IOException {
+    LOG.info("MOB compaction regular mode started");
+    baseTestMobFileCompaction();
+    LOG.info("MOB compaction regular mode finished OK");
+
+  }
+
+  @Override
+  protected void mobCompact(Admin admin, HTableDescriptor hdt, HColumnDescriptor hcd)
+      throws IOException, InterruptedException {
+    // Major compact MOB table
+    admin.majorCompact(hdt.getTableName(), hcd.getName());
+  }
+
+}
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
new file mode 100644
index 0000000..936173c
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularRegionBatchMode.java
@@ -0,0 +1,96 @@
+/**
+ *
+ * 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.mob;
+import java.io.IOException;
+
+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;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+  * Mob file compaction chore in a regular batch mode test.
+  * 1. Enables batch mode for regular MOB compaction,
+  *    Sets batch size to 7 regions.
+  * 2. Disables periodic MOB compactions, sets minimum age to archive to 10 sec
+  * 3. Creates MOB table with 20 regions
+  * 4. Loads MOB data (randomized keys, 1000 rows), flushes data.
+  * 5. Repeats 4. two more times
+  * 6. Verifies that we have 20 *3 = 60 mob files (equals to number of regions x 3)
+  * 7. Runs major MOB compaction.
+  * 8. Verifies that number of MOB files in a mob directory is 20 x4 = 80
+  * 9. Waits for a period of time larger than minimum age to archive
+  * 10. Runs Mob cleaner chore
+  * 11 Verifies that number of MOB files in a mob directory is 20.
+  * 12 Runs scanner and checks all 3 * 1000 rows.
+ */
+@SuppressWarnings("deprecation")
+@Category(LargeTests.class)
+public class TestMobCompactionRegularRegionBatchMode extends TestMobCompactionBase{
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestMobCompactionRegularRegionBatchMode.class);
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestMobCompactionRegularRegionBatchMode.class);
+  @Rule
+  public TestName testName = new TestName();
+
+  private int batchSize = 7;
+  private MobFileCompactionChore compactionChore;
+
+  public TestMobCompactionRegularRegionBatchMode() {
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    super.setUp();
+    compactionChore = new MobFileCompactionChore(conf, batchSize);
+  }
+
+  protected void initConf() {
+    super.initConf();
+    conf.setInt(MobConstants.MOB_MAJOR_COMPACTION_REGION_BATCH_SIZE, batchSize);
+  }
+
+  @Test
+  public void testMobFileCompactionBatchMode() throws InterruptedException, IOException {
+    LOG.info("MOB compaction chore regular batch mode started");
+    baseTestMobFileCompaction();
+    LOG.info("MOB compaction chore regular batch mode finished OK");
+
+  }
+
+  @Override
+  protected void mobCompact(Admin admin, HTableDescriptor hdt, HColumnDescriptor hcd)
+      throws IOException, InterruptedException {
+    // Major compact with batch mode enabled
+    compactionChore.performMajorCompactionInBatches(admin, hdt, hcd);
+  }
+
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileName.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileName.java
index ae53ff2..a6a2ee5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileName.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileName.java
@@ -47,6 +47,7 @@ public class TestMobFileName {
   private Date date;
   private String dateStr;
   private byte[] startKey;
+  private String regionName = "region";
 
   @Before
   public void setUp() {
@@ -59,32 +60,32 @@ public class TestMobFileName {
 
   @Test
   public void testHashCode() {
-    assertEquals(MobFileName.create(startKey, dateStr, uuid).hashCode(),
-        MobFileName.create(startKey, dateStr, uuid).hashCode());
-    assertNotSame(MobFileName.create(startKey, dateStr, uuid),
-        MobFileName.create(startKey, dateStr, uuid));
+    assertEquals(MobFileName.create(startKey, dateStr, uuid, regionName).hashCode(),
+        MobFileName.create(startKey, dateStr, uuid, regionName).hashCode());
+    assertNotSame(MobFileName.create(startKey, dateStr, uuid, regionName),
+        MobFileName.create(startKey, dateStr, uuid, regionName));
   }
 
   @Test
   public void testCreate() {
-    MobFileName mobFileName = MobFileName.create(startKey, dateStr, uuid);
+    MobFileName mobFileName = MobFileName.create(startKey, dateStr, uuid, regionName);
     assertEquals(mobFileName, MobFileName.create(mobFileName.getFileName()));
   }
 
   @Test
   public void testGet() {
-    MobFileName mobFileName = MobFileName.create(startKey, dateStr, uuid);
+    MobFileName mobFileName = MobFileName.create(startKey, dateStr, uuid, regionName);
     assertEquals(MD5Hash.getMD5AsHex(startKey, 0, startKey.length), mobFileName.getStartKey());
     assertEquals(dateStr, mobFileName.getDate());
     assertEquals(mobFileName.getFileName(), MD5Hash.getMD5AsHex(startKey, 0, startKey.length)
-        + dateStr + uuid);
+        + dateStr + uuid+"_"+regionName);
   }
 
   @Test
   public void testEquals() {
-    MobFileName mobFileName = MobFileName.create(startKey, dateStr, uuid);
+    MobFileName mobFileName = MobFileName.create(startKey, dateStr, uuid, regionName);
     assertTrue(mobFileName.equals(mobFileName));
     assertFalse(mobFileName.equals(this));
-    assertTrue(mobFileName.equals(MobFileName.create(startKey, dateStr, uuid)));
+    assertTrue(mobFileName.equals(MobFileName.create(startKey, dateStr, uuid, regionName)));
   }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestMobCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestMobCompactor.java
deleted file mode 100644
index b8e3ce0..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestMobCompactor.java
+++ /dev/null
@@ -1,1214 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.mob.compactions;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-import java.security.Key;
-import java.security.SecureRandom;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Calendar;
-import java.util.Collections;
-import java.util.List;
-import java.util.Objects;
-import java.util.Optional;
-import java.util.Random;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.RejectedExecutionException;
-import java.util.concurrent.RejectedExecutionHandler;
-import java.util.concurrent.SynchronousQueue;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-import javax.crypto.spec.SecretKeySpec;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.HBaseClassTestRule;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.NamespaceDescriptor;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.BufferedMutator;
-import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
-import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
-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.Delete;
-import org.apache.hadoop.hbase.client.Durability;
-import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.MobCompactPartitionPolicy;
-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.Scan;
-import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
-import org.apache.hadoop.hbase.coprocessor.ObserverContext;
-import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
-import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-import org.apache.hadoop.hbase.coprocessor.RegionObserver;
-import org.apache.hadoop.hbase.io.HFileLink;
-import org.apache.hadoop.hbase.io.crypto.KeyProviderForTesting;
-import org.apache.hadoop.hbase.io.crypto.aes.AES;
-import org.apache.hadoop.hbase.io.hfile.CacheConfig;
-import org.apache.hadoop.hbase.io.hfile.HFile;
-import org.apache.hadoop.hbase.master.cleaner.TimeToLiveHFileCleaner;
-import org.apache.hadoop.hbase.mob.MobConstants;
-import org.apache.hadoop.hbase.mob.MobFileName;
-import org.apache.hadoop.hbase.mob.MobTestUtil;
-import org.apache.hadoop.hbase.mob.MobUtils;
-import org.apache.hadoop.hbase.regionserver.BloomType;
-import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.HStoreFile;
-import org.apache.hadoop.hbase.regionserver.Store;
-import org.apache.hadoop.hbase.regionserver.StoreFile;
-import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
-import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
-import org.apache.hadoop.hbase.security.EncryptionUtil;
-import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.testclassification.LargeTests;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.Threads;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.ClassRule;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.rules.TestName;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-@Category(LargeTests.class)
-public class TestMobCompactor {
-
-  @ClassRule
-  public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestMobCompactor.class);
-
-  private static final Logger LOG = LoggerFactory.getLogger(TestMobCompactor.class);
-  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-  private static Configuration conf = null;
-  private TableName tableName;
-  private static Connection conn;
-  private BufferedMutator bufMut;
-  private Table table;
-  private static Admin admin;
-  private TableDescriptorBuilder desc;
-  private ColumnFamilyDescriptorBuilder cfdb1;
-  private ColumnFamilyDescriptorBuilder cfdb2;
-  private static FileSystem fs;
-  private static final String family1 = "family1";
-  private static final String family2 = "family2";
-  private static final String qf1 = "qualifier1";
-  private static final String qf2 = "qualifier2";
-
-  private static long tsFor20150907Monday;
-  private static long tsFor20151120Sunday;
-  private static long tsFor20151128Saturday;
-  private static long tsFor20151130Monday;
-  private static long tsFor20151201Tuesday;
-  private static long tsFor20151205Saturday;
-  private static long tsFor20151228Monday;
-  private static long tsFor20151231Thursday;
-  private static long tsFor20160101Friday;
-  private static long tsFor20160103Sunday;
-
-  private static final byte[] mobKey01 = Bytes.toBytes("r01");
-  private static final byte[] mobKey02 = Bytes.toBytes("r02");
-  private static final byte[] mobKey03 = Bytes.toBytes("r03");
-  private static final byte[] mobKey04 = Bytes.toBytes("r04");
-  private static final byte[] mobKey05 = Bytes.toBytes("r05");
-  private static final byte[] mobKey06 = Bytes.toBytes("r05");
-  private static final byte[] mobKey1 = Bytes.toBytes("r1");
-  private static final byte[] mobKey2 = Bytes.toBytes("r2");
-  private static final byte[] mobKey3 = Bytes.toBytes("r3");
-  private static final byte[] mobKey4 = Bytes.toBytes("r4");
-  private static final byte[] mobKey5 = Bytes.toBytes("r5");
-  private static final byte[] mobKey6 = Bytes.toBytes("r6");
-  private static final byte[] mobKey7 = Bytes.toBytes("r7");
-  private static final byte[] mobKey8 = Bytes.toBytes("r8");
-  private static final String mobValue0 = "mobValue00000000000000000000000000";
-  private static final String mobValue1 = "mobValue00000111111111111111111111";
-  private static final String mobValue2 = "mobValue00000222222222222222222222";
-  private static final String mobValue3 = "mobValue00000333333333333333333333";
-  private static final String mobValue4 = "mobValue00000444444444444444444444";
-  private static final String mobValue5 = "mobValue00000666666666666666666666";
-  private static final String mobValue6 = "mobValue00000777777777777777777777";
-  private static final String mobValue7 = "mobValue00000888888888888888888888";
-  private static final String mobValue8 = "mobValue00000888888888888888888899";
-
-  private static byte[] KEYS = Bytes.toBytes("012");
-  private static int regionNum = KEYS.length;
-  private static int delRowNum = 1;
-  private static int delCellNum = 6;
-  private static int cellNumPerRow = 3;
-  private static int rowNumPerFile = 2;
-  private static ExecutorService pool;
-
-  @Rule
-  public TestName name = new TestName();
-
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    TEST_UTIL.getConfiguration().setLong(MobConstants.MOB_COMPACTION_MERGEABLE_THRESHOLD, 5000);
-    TEST_UTIL.getConfiguration()
-        .set(HConstants.CRYPTO_KEYPROVIDER_CONF_KEY, KeyProviderForTesting.class.getName());
-    TEST_UTIL.getConfiguration().set(HConstants.CRYPTO_MASTERKEY_NAME_CONF_KEY, "hbase");
-    TEST_UTIL.getConfiguration().setLong(TimeToLiveHFileCleaner.TTL_CONF_KEY, 0);
-    TEST_UTIL.getConfiguration().setInt("hbase.client.retries.number", 1);
-    TEST_UTIL.getConfiguration().setInt("hbase.hfile.compaction.discharger.interval", 100);
-    TEST_UTIL.startMiniCluster(1);
-    pool = createThreadPool(TEST_UTIL.getConfiguration());
-    conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration(), pool);
-    fs = TEST_UTIL.getTestFileSystem();
-    conf = TEST_UTIL.getConfiguration();
-    admin = TEST_UTIL.getAdmin();
-
-    // Initialize timestamps for these days
-    Calendar calendar =  Calendar.getInstance();
-    calendar.set(2015, 8, 7, 10, 20);
-    tsFor20150907Monday = calendar.getTimeInMillis();
-
-    calendar.set(2015, 10, 20, 10, 20);
-    tsFor20151120Sunday = calendar.getTimeInMillis();
-
-    calendar.set(2015, 10, 28, 10, 20);
-    tsFor20151128Saturday = calendar.getTimeInMillis();
-
-    calendar.set(2015, 10, 30, 10, 20);
-    tsFor20151130Monday = calendar.getTimeInMillis();
-
-    calendar.set(2015, 11, 1, 10, 20);
-    tsFor20151201Tuesday = calendar.getTimeInMillis();
-
-    calendar.set(2015, 11, 5, 10, 20);
-    tsFor20151205Saturday = calendar.getTimeInMillis();
-
-    calendar.set(2015, 11, 28, 10, 20);
-    tsFor20151228Monday = calendar.getTimeInMillis();
-
-    calendar.set(2015, 11, 31, 10, 20);
-    tsFor20151231Thursday = calendar.getTimeInMillis();
-
-    calendar.set(2016, 0, 1, 10, 20);
-    tsFor20160101Friday = calendar.getTimeInMillis();
-
-    calendar.set(2016, 0, 3, 10, 20);
-    tsFor20160103Sunday = calendar.getTimeInMillis();
-  }
-
-  @AfterClass
-  public static void tearDownAfterClass() throws Exception {
-    pool.shutdown();
-    conn.close();
-    TEST_UTIL.shutdownMiniCluster();
-  }
-
-  public void setUp(String tableNameAsString) throws IOException {
-    tableName = TableName.valueOf(tableNameAsString);
-    cfdb1 = ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(family1));
-    cfdb1.setMobEnabled(true);
-    cfdb1.setMobThreshold(5);
-    cfdb2 = ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(family2));
-    cfdb2.setMobEnabled(true);
-    cfdb2.setMobThreshold(5);
-    desc = TableDescriptorBuilder.newBuilder(tableName);
-    desc.setColumnFamily(cfdb1.build());
-    desc.setColumnFamily(cfdb2.build());
-    admin.createTable(desc.build(), getSplitKeys());
-    table = conn.getTable(tableName);
-    bufMut = conn.getBufferedMutator(tableName);
-  }
-
-  // Set up for mob compaction policy testing
-  private void setUpForPolicyTest(String tableNameAsString, MobCompactPartitionPolicy type)
-      throws IOException {
-    tableName = TableName.valueOf(tableNameAsString);
-    cfdb1 = ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(family1));
-    cfdb1.setMobEnabled(true);
-    cfdb1.setMobThreshold(10);
-    cfdb1.setMobCompactPartitionPolicy(type);
-    desc = TableDescriptorBuilder.newBuilder(tableName);
-    desc.setColumnFamily(cfdb1.build());
-    admin.createTable(desc.build());
-    table = conn.getTable(tableName);
-    bufMut = conn.getBufferedMutator(tableName);
-  }
-
-  // alter mob compaction policy
-  private void alterForPolicyTest(final MobCompactPartitionPolicy type)
-      throws Exception {
-
-    cfdb1.setMobCompactPartitionPolicy(type);
-    desc.modifyColumnFamily(cfdb1.build());
-    admin.modifyTable(desc.build());
-    LOG.info("alter status finished");
-  }
-
-  @Test
-  public void testMinorCompaction() throws Exception {
-    resetConf();
-    int mergeSize = 5000;
-    // change the mob compaction merge size
-    conf.setLong(MobConstants.MOB_COMPACTION_MERGEABLE_THRESHOLD, mergeSize);
-
-    // create a table with namespace
-    NamespaceDescriptor namespaceDescriptor = NamespaceDescriptor.create("ns").build();
-    String tableNameAsString = "ns:testMinorCompaction";
-    admin.createNamespace(namespaceDescriptor);
-    setUp(tableNameAsString);
-    int count = 4;
-    // generate mob files
-    loadData(admin, bufMut, tableName, count, rowNumPerFile);
-    int rowNumPerRegion = count * rowNumPerFile;
-
-    assertEquals("Before deleting: mob rows count", regionNum * rowNumPerRegion,
-      MobTestUtil.countMobRows(table));
-    assertEquals("Before deleting: mob cells count", regionNum * cellNumPerRow * rowNumPerRegion,
-      countMobCells(table));
-    assertEquals("Before deleting: mob file count", regionNum * count,
-      countFiles(tableName, true, family1));
-
-    int largeFilesCount = countLargeFiles(mergeSize, tableName, family1);
-    createDelFile(table, tableName, Bytes.toBytes(family1), Bytes.toBytes(qf1));
-
-    assertEquals("Before compaction: mob rows count", regionNum * (rowNumPerRegion - delRowNum),
-      MobTestUtil.countMobRows(table));
-    assertEquals("Before compaction: mob cells count", regionNum
-      * (cellNumPerRow * rowNumPerRegion - delCellNum), countMobCells(table));
-    assertEquals("Before compaction: family1 mob file count", regionNum * count,
-      countFiles(tableName, true, family1));
-    assertEquals("Before compaction: family2 mob file count", regionNum * count,
-      countFiles(tableName, true, family2));
-    assertEquals("Before compaction: family1 del file count", regionNum,
-      countFiles(tableName, false, family1));
-    assertEquals("Before compaction: family2 del file count", regionNum,
-      countFiles(tableName, false, family2));
-
-    // do the mob file compaction
-    MobCompactor compactor = new PartitionedMobCompactor(conf, fs, tableName,
-      cfdb1.build(), pool);
-    compactor.compact();
-
-    assertEquals("After compaction: mob rows count", regionNum * (rowNumPerRegion - delRowNum),
-      MobTestUtil.countMobRows(table));
-    assertEquals("After compaction: mob cells count", regionNum
-      * (cellNumPerRow * rowNumPerRegion - delCellNum), countMobCells(table));
-    // After the compaction, the files smaller than the mob compaction merge size
-    // is merge to one file
-    assertEquals("After compaction: family1 mob file count", largeFilesCount + regionNum,
-      countFiles(tableName, true, family1));
-    assertEquals("After compaction: family2 mob file count", regionNum * count,
-      countFiles(tableName, true, family2));
-    assertEquals("After compaction: family1 del file count", regionNum,
-      countFiles(tableName, false, family1));
-    assertEquals("After compaction: family2 del file count", regionNum,
-      countFiles(tableName, false, family2));
-  }
-
-  @Test
-  public void testMinorCompactionWithWeeklyPolicy() throws Exception {
-    resetConf();
-    int mergeSize = 5000;
-    // change the mob compaction merge size
-    conf.setLong(MobConstants.MOB_COMPACTION_MERGEABLE_THRESHOLD, mergeSize);
-
-    commonPolicyTestLogic("testMinorCompactionWithWeeklyPolicy",
-        MobCompactPartitionPolicy.WEEKLY, false, 6,
-        new String[] { "20150907", "20151120", "20151128", "20151130", "20151205", "20160103" },
-        true);
-  }
-
-  @Test
-  public void testMajorCompactionWithWeeklyPolicy() throws Exception {
-    resetConf();
-
-    commonPolicyTestLogic("testMajorCompactionWithWeeklyPolicy",
-        MobCompactPartitionPolicy.WEEKLY, true, 5,
-        new String[] { "20150907", "20151120", "20151128", "20151205", "20160103" }, true);
-  }
-
-  @Test
-  public void testMinorCompactionWithMonthlyPolicy() throws Exception {
-    resetConf();
-    int mergeSize = 5000;
-    // change the mob compaction merge size
-    conf.setLong(MobConstants.MOB_COMPACTION_MERGEABLE_THRESHOLD, mergeSize);
-
-    commonPolicyTestLogic("testMinorCompactionWithMonthlyPolicy",
-        MobCompactPartitionPolicy.MONTHLY, false, 4,
-        new String[] { "20150907", "20151130", "20151231", "20160103" }, true);
-  }
-
-  @Test
-  public void testMajorCompactionWithMonthlyPolicy() throws Exception {
-    resetConf();
-
-    commonPolicyTestLogic("testMajorCompactionWithMonthlyPolicy",
-        MobCompactPartitionPolicy.MONTHLY, true, 4,
-        new String[] {"20150907", "20151130", "20151231", "20160103"}, true);
-  }
-
-  @Test
-  public void testMajorCompactionWithWeeklyFollowedByMonthly() throws Exception {
-    resetConf();
-
-    commonPolicyTestLogic("testMajorCompactionWithWeeklyFollowedByMonthly",
-        MobCompactPartitionPolicy.WEEKLY, true, 5,
-        new String[] { "20150907", "20151120", "20151128", "20151205", "20160103" }, true);
-
-    commonPolicyTestLogic("testMajorCompactionWithWeeklyFollowedByMonthly",
-        MobCompactPartitionPolicy.MONTHLY, true, 4,
-        new String[] {"20150907", "20151128", "20151205", "20160103" }, false);
-  }
-
-  @Test
-  public void testMajorCompactionWithWeeklyFollowedByMonthlyFollowedByWeekly() throws Exception {
-    resetConf();
-
-    commonPolicyTestLogic("testMajorCompactionWithWeeklyFollowedByMonthlyFollowedByWeekly",
-        MobCompactPartitionPolicy.WEEKLY, true, 5,
-        new String[] { "20150907", "20151120", "20151128", "20151205", "20160103" }, true);
-
-    commonPolicyTestLogic("testMajorCompactionWithWeeklyFollowedByMonthlyFollowedByWeekly",
-        MobCompactPartitionPolicy.MONTHLY, true, 4,
-        new String[] { "20150907", "20151128", "20151205", "20160103" }, false);
-
-    commonPolicyTestLogic("testMajorCompactionWithWeeklyFollowedByMonthlyFollowedByWeekly",
-        MobCompactPartitionPolicy.WEEKLY, true, 4,
-        new String[] { "20150907", "20151128", "20151205", "20160103" }, false);
-  }
-
-  @Test
-  public void testMajorCompactionWithWeeklyFollowedByMonthlyFollowedByDaily() throws Exception {
-    resetConf();
-
-    commonPolicyTestLogic("testMajorCompactionWithWeeklyFollowedByMonthlyFollowedByDaily",
-        MobCompactPartitionPolicy.WEEKLY, true, 5,
-        new String[] { "20150907", "20151120", "20151128", "20151205", "20160103" }, true);
-
-    commonPolicyTestLogic("testMajorCompactionWithWeeklyFollowedByMonthlyFollowedByDaily",
-        MobCompactPartitionPolicy.MONTHLY, true, 4,
-        new String[] { "20150907", "20151128", "20151205", "20160103" }, false);
-
-    commonPolicyTestLogic("testMajorCompactionWithWeeklyFollowedByMonthlyFollowedByDaily",
-        MobCompactPartitionPolicy.DAILY, true, 4,
-        new String[] { "20150907", "20151128", "20151205", "20160103" }, false);
-  }
-
-  @Test
-  public void testCompactionWithHFileLink() throws IOException, InterruptedException {
-    resetConf();
-    String tableNameAsString = "testCompactionWithHFileLink";
-    setUp(tableNameAsString);
-    int count = 4;
-    // generate mob files
-    loadData(admin, bufMut, tableName, count, rowNumPerFile);
-    int rowNumPerRegion = count * rowNumPerFile;
-
-    long tid = System.currentTimeMillis();
-    String snapshotName1 = "snaptb-" + tid;
-    // take a snapshot
-    admin.snapshot(snapshotName1, tableName);
-
-    createDelFile(table, tableName, Bytes.toBytes(family1), Bytes.toBytes(qf1));
-
-    assertEquals("Before compaction: mob rows count", regionNum * (rowNumPerRegion - delRowNum),
-      MobTestUtil.countMobRows(table));
-    assertEquals("Before compaction: mob cells count", regionNum
-      * (cellNumPerRow * rowNumPerRegion - delCellNum), countMobCells(table));
-    assertEquals("Before compaction: family1 mob file count", regionNum * count,
-      countFiles(tableName, true, family1));
-    assertEquals("Before compaction: family2 mob file count", regionNum * count,
-      countFiles(tableName, true, family2));
-    assertEquals("Before compaction: family1 del file count", regionNum,
-      countFiles(tableName, false, family1));
-    assertEquals("Before compaction: family2 del file count", regionNum,
-      countFiles(tableName, false, family2));
-
-    // do the mob compaction
-    MobCompactor compactor = new PartitionedMobCompactor(conf, fs, tableName,
-      cfdb1.build(), pool);
-    compactor.compact();
-
-    assertEquals("After first compaction: mob rows count", regionNum
-      * (rowNumPerRegion - delRowNum), MobTestUtil.countMobRows(table));
-    assertEquals("After first compaction: mob cells count", regionNum
-      * (cellNumPerRow * rowNumPerRegion - delCellNum), countMobCells(table));
-    assertEquals("After first compaction: family1 mob file count", regionNum,
-      countFiles(tableName, true, family1));
-    assertEquals("After first compaction: family2 mob file count", regionNum * count,
-      countFiles(tableName, true, family2));
-    assertEquals("After first compaction: family1 del file count", 0,
-      countFiles(tableName, false, family1));
-    assertEquals("After first compaction: family2 del file count", regionNum,
-      countFiles(tableName, false, family2));
-    assertEquals("After first compaction: family1 hfilelink count", 0, countHFileLinks(family1));
-    assertEquals("After first compaction: family2 hfilelink count", 0, countHFileLinks(family2));
-
-    admin.disableTable(tableName);
-    // Restore from snapshot, the hfilelink will exist in mob dir
-    admin.restoreSnapshot(snapshotName1);
-    admin.enableTable(tableName);
-
-    assertEquals("After restoring snapshot: mob rows count", regionNum * rowNumPerRegion,
-      MobTestUtil.countMobRows(table));
-    assertEquals("After restoring snapshot: mob cells count", regionNum * cellNumPerRow
-      * rowNumPerRegion, countMobCells(table));
-    assertEquals("After restoring snapshot: family1 mob file count", regionNum * count,
-      countFiles(tableName, true, family1));
-    assertEquals("After restoring snapshot: family2 mob file count", regionNum * count,
-      countFiles(tableName, true, family2));
-    assertEquals("After restoring snapshot: family1 del file count", 0,
-      countFiles(tableName, false, family1));
-    assertEquals("After restoring snapshot: family2 del file count", 0,
-      countFiles(tableName, false, family2));
-    assertEquals("After restoring snapshot: family1 hfilelink count", regionNum * count,
-      countHFileLinks(family1));
-    assertEquals("After restoring snapshot: family2 hfilelink count", 0, countHFileLinks(family2));
-
-    compactor.compact();
-
-    assertEquals("After second compaction: mob rows count", regionNum * rowNumPerRegion,
-      MobTestUtil.countMobRows(table));
-    assertEquals("After second compaction: mob cells count", regionNum * cellNumPerRow
-      * rowNumPerRegion, countMobCells(table));
-    assertEquals("After second compaction: family1 mob file count", regionNum,
-      countFiles(tableName, true, family1));
-    assertEquals("After second compaction: family2 mob file count", regionNum * count,
-      countFiles(tableName, true, family2));
-    assertEquals("After second compaction: family1 del file count", 0,
-      countFiles(tableName, false, family1));
-    assertEquals("After second compaction: family2 del file count", 0,
-      countFiles(tableName, false, family2));
-    assertEquals("After second compaction: family1 hfilelink count", 0, countHFileLinks(family1));
-    assertEquals("After second compaction: family2 hfilelink count", 0, countHFileLinks(family2));
-    assertRefFileNameEqual(family1);
-  }
-
-  @Test
-  public void testMajorCompactionFromAdmin() throws Exception {
-    resetConf();
-    int mergeSize = 5000;
-    // change the mob compaction merge size
-    conf.setLong(MobConstants.MOB_COMPACTION_MERGEABLE_THRESHOLD, mergeSize);
-    SecureRandom rng = new SecureRandom();
-    byte[] keyBytes = new byte[AES.KEY_LENGTH];
-    rng.nextBytes(keyBytes);
-    String algorithm = conf.get(HConstants.CRYPTO_KEY_ALGORITHM_CONF_KEY, HConstants.CIPHER_AES);
-    Key cfKey = new SecretKeySpec(keyBytes, algorithm);
-    byte[] encryptionKey = EncryptionUtil.wrapKey(conf,
-      conf.get(HConstants.CRYPTO_MASTERKEY_NAME_CONF_KEY, User.getCurrent().getShortName()), cfKey);
-    final TableName tableName = TableName.valueOf(name.getMethodName());
-    TableDescriptorBuilder tableDescriptorBuilder =
-      TableDescriptorBuilder.newBuilder(tableName);
-    ColumnFamilyDescriptor cfd1 =
-      ColumnFamilyDescriptorBuilder
-        .newBuilder(Bytes.toBytes(family1))
-        .setMobEnabled(true)
-        .setMobThreshold(0)
-        .setEncryptionType(algorithm)
-        .setEncryptionKey(encryptionKey).build();
-    ColumnFamilyDescriptor cfd2 =
-      ColumnFamilyDescriptorBuilder
-        .newBuilder(Bytes.toBytes(family2))
-        .setMobEnabled(true)
-        .setMobThreshold(0).build();
-    tableDescriptorBuilder.setColumnFamily(cfd1);
-    tableDescriptorBuilder.setColumnFamily(cfd2);
-    admin.createTable(tableDescriptorBuilder.build(), getSplitKeys());
-    Table table = conn.getTable(tableName);
-    BufferedMutator bufMut = conn.getBufferedMutator(tableName);
-    int count = 4;
-    // generate mob files
-    loadData(admin, bufMut, tableName, count, rowNumPerFile);
-    int rowNumPerRegion = count * rowNumPerFile;
-
-    assertEquals("Before deleting: mob rows count", regionNum * rowNumPerRegion,
-      MobTestUtil.countMobRows(table));
-    assertEquals("Before deleting: mob cells count", regionNum * cellNumPerRow * rowNumPerRegion,
-      countMobCells(table));
-    assertEquals("Before deleting: mob file count", regionNum * count,
-      countFiles(tableName, true, family1));
-
-    createDelFile(table, tableName, Bytes.toBytes(family1), Bytes.toBytes(qf1));
-
-    assertEquals("Before compaction: mob rows count", regionNum * (rowNumPerRegion - delRowNum),
-      MobTestUtil.countMobRows(table));
-    assertEquals("Before compaction: mob cells count", regionNum
-      * (cellNumPerRow * rowNumPerRegion - delCellNum), countMobCells(table));
-    assertEquals("Before compaction: family1 mob file count", regionNum * count,
-      countFiles(tableName, true, family1));
-    assertEquals("Before compaction: family2 mob file count", regionNum * count,
-      countFiles(tableName, true, family2));
-    assertEquals("Before compaction: family1 del file count", regionNum,
-      countFiles(tableName, false, family1));
-    assertEquals("Before compaction: family2 del file count", regionNum,
-      countFiles(tableName, false, family2));
-
-    // do the major mob compaction, it will force all files to compaction
-    admin.majorCompact(tableName, cfd1.getName(), CompactType.MOB);
-
-    waitUntilMobCompactionFinished(tableName);
-    assertEquals("After compaction: mob rows count", regionNum * (rowNumPerRegion - delRowNum),
-      MobTestUtil.countMobRows(table));
-    assertEquals("After compaction: mob cells count", regionNum
-      * (cellNumPerRow * rowNumPerRegion - delCellNum), countMobCells(table));
-    assertEquals("After compaction: family1 mob file count", regionNum,
-      countFiles(tableName, true, family1));
-    assertEquals("After compaction: family2 mob file count", regionNum * count,
-      countFiles(tableName, true, family2));
-    assertEquals("After compaction: family1 del file count", 0,
-      countFiles(tableName, false, family1));
-    assertEquals("After compaction: family2 del file count", regionNum,
-      countFiles(tableName, false, family2));
-    Assert.assertTrue(verifyEncryption(tableName, family1));
-    table.close();
-  }
-
-  @Test
-  public void testScannerOnBulkLoadRefHFiles() throws Exception {
-    resetConf();
-    setUp("testScannerOnBulkLoadRefHFiles");
-    long ts = EnvironmentEdgeManager.currentTime();
-    byte[] key0 = Bytes.toBytes("k0");
-    byte[] key1 = Bytes.toBytes("k1");
-    String value0 = "mobValue0";
-    String value1 = "mobValue1";
-    String newValue0 = "new";
-    Put put0 = new Put(key0);
-    put0.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), ts, Bytes.toBytes(value0));
-    loadData(admin, bufMut, tableName, new Put[] { put0 });
-    put0 = new Put(key0);
-    put0.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), ts, Bytes.toBytes(newValue0));
-    Put put1 = new Put(key1);
-    put1.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), ts, Bytes.toBytes(value1));
-    loadData(admin, bufMut, tableName, new Put[] { put0, put1 });
-    // read the latest cell of key0.
-    Get get = new Get(key0);
-    Result result = table.get(get);
-    ColumnFamilyDescriptor cfd1 = cfdb1.build();
-    Cell cell = result.getColumnLatestCell(cfd1.getName(), Bytes.toBytes(qf1));
-    assertEquals("Before compaction: mob value of k0", newValue0,
-      Bytes.toString(CellUtil.cloneValue(cell)));
-    admin.majorCompact(tableName, cfd1.getName(), CompactType.MOB);
-    waitUntilMobCompactionFinished(tableName);
-    // read the latest cell of key0, the cell seqId in bulk loaded file is not reset in the
-    // scanner. The cell that has "new" value is still visible.
-    result = table.get(get);
-    cell = result.getColumnLatestCell(cfd1.getName(), Bytes.toBytes(qf1));
-    assertEquals("After compaction: mob value of k0", newValue0,
-      Bytes.toString(CellUtil.cloneValue(cell)));
-    // read the ref cell, not read further to the mob cell.
-    get = new Get(key1);
-    get.setAttribute(MobConstants.MOB_SCAN_RAW, Bytes.toBytes(true));
-    result = table.get(get);
-    cell = result.getColumnLatestCell(cfd1.getName(), Bytes.toBytes(qf1));
-    // the ref name is the new file
-    Path mobFamilyPath =
-      MobUtils.getMobFamilyPath(TEST_UTIL.getConfiguration(), tableName,
-        cfdb1.getNameAsString());
-    List<Path> paths = new ArrayList<>();
-    if (fs.exists(mobFamilyPath)) {
-      FileStatus[] files = fs.listStatus(mobFamilyPath);
-      for (FileStatus file : files) {
-        if (!StoreFileInfo.isDelFile(file.getPath())) {
-          paths.add(file.getPath());
-        }
-      }
-    }
-    assertEquals("After compaction: number of mob files:", 1, paths.size());
-    assertEquals("After compaction: mob file name:", MobUtils.getMobFileName(cell), paths.get(0)
-      .getName());
-  }
-
-  /**
-   * This case tests the following mob compaction and normal compaction scenario,
-   * after mob compaction, the mob reference in new bulkloaded hfile will win even after it
-   * is compacted with some other normal hfiles. This is to make sure the mvcc is included
-   * after compaction for mob enabled store files.
-   */
-  @Test
-  public void testGetAfterCompaction() throws Exception {
-    resetConf();
-    conf.setLong(TimeToLiveHFileCleaner.TTL_CONF_KEY, 0);
-    String famStr = "f1";
-    byte[] fam = Bytes.toBytes(famStr);
-    byte[] qualifier = Bytes.toBytes("q1");
-    byte[] mobVal = Bytes.toBytes("01234567890");
-    HTableDescriptor hdt = new HTableDescriptor(TableName.valueOf(name.getMethodName()));
-    hdt.addCoprocessor(CompactTwoLatestHfilesCopro.class.getName());
-    HColumnDescriptor hcd = new HColumnDescriptor(fam);
-    hcd.setMobEnabled(true);
-    hcd.setMobThreshold(10);
-    hcd.setMaxVersions(1);
-    hdt.addFamily(hcd);
-    try {
-      Table table = TEST_UTIL.createTable(hdt, null);
-      HRegion r = TEST_UTIL.getMiniHBaseCluster().getRegions(hdt.getTableName()).get(0);
-      Put p = new Put(Bytes.toBytes("r1"));
-      p.addColumn(fam, qualifier, mobVal);
-      table.put(p);
-      // Create mob file mob1 and reference file ref1
-      TEST_UTIL.flush(table.getName());
-      // Make sure that it is flushed.
-      FileSystem fs = r.getRegionFileSystem().getFileSystem();
-      Path path = r.getRegionFileSystem().getStoreDir(famStr);
-      waitUntilFilesShowup(fs, path, 1);
-
-      p = new Put(Bytes.toBytes("r2"));
-      p.addColumn(fam, qualifier, mobVal);
-      table.put(p);
-      // Create mob file mob2 and reference file ref2
-      TEST_UTIL.flush(table.getName());
-      waitUntilFilesShowup(fs, path, 2);
-      // Do mob compaction to create mob3 and ref3
-      TEST_UTIL.getAdmin().compact(hdt.getTableName(), fam, CompactType.MOB);
-      waitUntilFilesShowup(fs, path, 3);
-
-      // Compact ref3 and ref2 into ref4
-      TEST_UTIL.getAdmin().compact(hdt.getTableName(), fam);
-      waitUntilFilesShowup(fs, path, 2);
-
-      // Sleep for some time, since TimeToLiveHFileCleaner is 0, the next run of
-      // clean chore is guaranteed to clean up files in archive
-      Thread.sleep(100);
-      // Run cleaner to make sure that files in archive directory are cleaned up
-      TEST_UTIL.getMiniHBaseCluster().getMaster().getHFileCleaner().choreForTesting();
-
-      // Get "r2"
-      Get get = new Get(Bytes.toBytes("r2"));
-      try {
-        Result result = table.get(get);
-        assertTrue(Arrays.equals(result.getValue(fam, qualifier), mobVal));
-      } catch (IOException e) {
-        assertTrue("The MOB file doesn't exist", false);
-      }
-    } finally {
-      TEST_UTIL.deleteTable(hdt.getTableName());
-    }
-  }
-
-  private void waitUntilFilesShowup(final FileSystem fs, final Path path, final int num)
-    throws InterruptedException, IOException {
-    FileStatus[] fileList = fs.listStatus(path);
-    while (fileList.length != num) {
-      Thread.sleep(50);
-      fileList = fs.listStatus(path);
-      for (FileStatus fileStatus: fileList) {
-        LOG.info(Objects.toString(fileStatus));
-      }
-    }
-  }
-
-  /**
-   * This copro overwrites the default compaction policy. It always chooses two latest hfiles and
-   * compacts them into a new one.
-   */
-  public static class CompactTwoLatestHfilesCopro implements RegionCoprocessor, RegionObserver {
-
-    @Override
-    public Optional<RegionObserver> getRegionObserver() {
-      return Optional.of(this);
-    }
-
-    @Override
-    public void preCompactSelection(ObserverContext<RegionCoprocessorEnvironment> c, Store store,
-        List<? extends StoreFile> candidates, CompactionLifeCycleTracker tracker)
-        throws IOException {
-      int count = candidates.size();
-      if (count >= 2) {
-        for (int i = 0; i < count - 2; i++) {
-          candidates.remove(0);
-        }
-        c.bypass();
-      }
-    }
-  }
-
-  private void waitUntilMobCompactionFinished(TableName tableName) throws IOException,
-    InterruptedException {
-    long finished = EnvironmentEdgeManager.currentTime() + 60000;
-    CompactionState state = admin.getCompactionState(tableName, CompactType.MOB);
-    while (EnvironmentEdgeManager.currentTime() < finished) {
-      if (state == CompactionState.NONE) {
-        break;
-      }
-      state = admin.getCompactionState(tableName, CompactType.MOB);
-      Thread.sleep(10);
-    }
-    assertEquals(CompactionState.NONE, state);
-  }
-
-  /**
-   * Gets the number of cells in the given table.
-   * @param table to get the  scanner
-   * @return the number of cells
-   */
-  private int countMobCells(final Table table) throws IOException {
-    Scan scan = new Scan();
-    // Do not retrieve the mob data when scanning
-    scan.setAttribute(MobConstants.MOB_SCAN_RAW, Bytes.toBytes(Boolean.TRUE));
-    ResultScanner results = table.getScanner(scan);
-    int count = 0;
-    for (Result res : results) {
-      count += res.size();
-    }
-    results.close();
-    return count;
-  }
-
-  /**
-   * Gets the number of files in the mob path.
-   * @param isMobFile gets number of the mob files or del files
-   * @param familyName the family name
-   * @return the number of the files
-   */
-  private int countFiles(TableName tableName, boolean isMobFile, String familyName)
-    throws IOException {
-    Path mobDirPath = MobUtils.getMobFamilyPath(conf, tableName, familyName);
-    int count = 0;
-    if (fs.exists(mobDirPath)) {
-      FileStatus[] files = fs.listStatus(mobDirPath);
-      for (FileStatus file : files) {
-        if (isMobFile == true) {
-          if (!StoreFileInfo.isDelFile(file.getPath())) {
-            count++;
-          }
-        } else {
-          if (StoreFileInfo.isDelFile(file.getPath())) {
-            count++;
-          }
-        }
-      }
-    }
-    return count;
-  }
-
-  private boolean verifyEncryption(TableName tableName, String familyName) throws IOException {
-    Path mobDirPath = MobUtils.getMobFamilyPath(conf, tableName, familyName);
-    boolean hasFiles = false;
-    if (fs.exists(mobDirPath)) {
-      FileStatus[] files = fs.listStatus(mobDirPath);
-      hasFiles = files != null && files.length > 0;
-      Assert.assertTrue(hasFiles);
-      Path path = files[0].getPath();
-      CacheConfig cacheConf = new CacheConfig(conf);
-      HStoreFile sf = new HStoreFile(TEST_UTIL.getTestFileSystem(), path, conf, cacheConf,
-        BloomType.NONE, true);
-      sf.initReader();
-      HFile.Reader reader = sf.getReader().getHFileReader();
-      byte[] encryptionKey = reader.getTrailer().getEncryptionKey();
-      Assert.assertTrue(null != encryptionKey);
-      Assert.assertTrue(reader.getFileContext().getEncryptionContext().getCipher().getName()
-        .equals(HConstants.CIPHER_AES));
-    }
-    return hasFiles;
-  }
-
-  /**
-   * Gets the number of HFileLink in the mob path.
-   * @param familyName the family name
-   * @return the number of the HFileLink
-   */
-  private int countHFileLinks(String familyName) throws IOException {
-    Path mobDirPath = MobUtils.getMobFamilyPath(conf, tableName, familyName);
-    int count = 0;
-    if (fs.exists(mobDirPath)) {
-      FileStatus[] files = fs.listStatus(mobDirPath);
-      for (FileStatus file : files) {
-        if (HFileLink.isHFileLink(file.getPath())) {
-          count++;
-        }
-      }
-    }
-    return count;
-  }
-
-  /**
-   * Gets the number of files.
-   * @param size the size of the file
-   * @param tableName the current table name
-   * @param familyName the family name
-   * @return the number of files large than the size
-   */
-  private int countLargeFiles(int size, TableName tableName, String familyName) throws IOException {
-    Path mobDirPath = MobUtils.getMobFamilyPath(conf, tableName, familyName);
-    int count = 0;
-    if (fs.exists(mobDirPath)) {
-      FileStatus[] files = fs.listStatus(mobDirPath);
-      for (FileStatus file : files) {
-        // ignore the del files in the mob path
-        if ((!StoreFileInfo.isDelFile(file.getPath())) && (file.getLen() > size)) {
-          count++;
-        }
-      }
-    }
-    return count;
-  }
-
-  /**
-   * loads some data to the table.
-   */
-  private void loadData(Admin admin, BufferedMutator table, TableName tableName, int fileNum,
-    int rowNumPerFile) throws IOException, InterruptedException {
-    if (fileNum <= 0) {
-      throw new IllegalArgumentException();
-    }
-    for (int i = 0; i < fileNum * rowNumPerFile; i++) {
-      for (byte k0 : KEYS) {
-        byte[] k = new byte[] { k0 };
-        byte[] key = Bytes.add(k, Bytes.toBytes(i));
-        byte[] mobVal = makeDummyData(10 * (i + 1));
-        Put put = new Put(key);
-        put.setDurability(Durability.SKIP_WAL);
-        put.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), mobVal);
-        put.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf2), mobVal);
-        put.addColumn(Bytes.toBytes(family2), Bytes.toBytes(qf1), mobVal);
-        table.mutate(put);
-      }
-      if ((i + 1) % rowNumPerFile == 0) {
-        table.flush();
-        admin.flush(tableName);
-      }
-    }
-  }
-
-  private void loadData(Admin admin, BufferedMutator table, TableName tableName, Put[] puts)
-    throws IOException {
-    table.mutate(Arrays.asList(puts));
-    table.flush();
-    admin.flush(tableName);
-  }
-
-  private void loadDataForPartitionPolicy(Admin admin, BufferedMutator table, TableName tableName)
-      throws IOException {
-
-    Put[] pArray = new Put[1000];
-
-    for (int i = 0; i < 1000; i ++) {
-      Put put0 = new Put(Bytes.toBytes("r0" + i));
-      put0.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1),
-              tsFor20151130Monday, Bytes.toBytes(mobValue0));
-      pArray[i] = put0;
-    }
-    loadData(admin, bufMut, tableName, pArray);
-
-    Put put06 = new Put(mobKey06);
-    put06.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1),
-            tsFor20151128Saturday, Bytes.toBytes(mobValue0));
-
-    loadData(admin, bufMut, tableName, new Put[] { put06 });
-
-    Put put1 = new Put(mobKey1);
-    put1.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), tsFor20151201Tuesday,
-        Bytes.toBytes(mobValue1));
-    loadData(admin, bufMut, tableName, new Put[] { put1 });
-
-    Put put2 = new Put(mobKey2);
-    put2.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), tsFor20151205Saturday,
-        Bytes.toBytes(mobValue2));
-    loadData(admin, bufMut, tableName, new Put[] { put2 });
-
-    Put put3 = new Put(mobKey3);
-    put3.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), tsFor20151228Monday,
-        Bytes.toBytes(mobValue3));
-    loadData(admin, bufMut, tableName, new Put[] { put3 });
-
-    Put put4 = new Put(mobKey4);
-    put4.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), tsFor20151231Thursday,
-        Bytes.toBytes(mobValue4));
-    loadData(admin, bufMut, tableName, new Put[] { put4 });
-
-    Put put5 = new Put(mobKey5);
-    put5.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), tsFor20160101Friday,
-        Bytes.toBytes(mobValue5));
-    loadData(admin, bufMut, tableName, new Put[] { put5 });
-
-    Put put6 = new Put(mobKey6);
-    put6.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), tsFor20160103Sunday,
-        Bytes.toBytes(mobValue6));
-    loadData(admin, bufMut, tableName, new Put[] { put6 });
-
-    Put put7 = new Put(mobKey7);
-    put7.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), tsFor20150907Monday,
-        Bytes.toBytes(mobValue7));
-    loadData(admin, bufMut, tableName, new Put[] { put7 });
-
-    Put put8 = new Put(mobKey8);
-    put8.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), tsFor20151120Sunday,
-        Bytes.toBytes(mobValue8));
-    loadData(admin, bufMut, tableName, new Put[] { put8 });
-  }
-
-
-  /**
-   * delete the row, family and cell to create the del file
-   */
-  private void createDelFile(Table table, TableName tableName, byte[] family, byte[] qf)
-    throws IOException, InterruptedException {
-    for (byte k0 : KEYS) {
-      byte[] k = new byte[] { k0 };
-      // delete a family
-      byte[] key1 = Bytes.add(k, Bytes.toBytes(0));
-      Delete delete1 = new Delete(key1);
-      delete1.addFamily(family);
-      table.delete(delete1);
-      // delete one row
-      byte[] key2 = Bytes.add(k, Bytes.toBytes(2));
-      Delete delete2 = new Delete(key2);
-      table.delete(delete2);
-      // delete one cell
-      byte[] key3 = Bytes.add(k, Bytes.toBytes(4));
-      Delete delete3 = new Delete(key3);
-      delete3.addColumn(family, qf);
-      table.delete(delete3);
-    }
-    admin.flush(tableName);
-    List<HRegion> regions = TEST_UTIL.getHBaseCluster().getRegions(tableName);
-    for (HRegion region : regions) {
-      region.waitForFlushesAndCompactions();
-      region.compact(true);
-    }
-  }
-  /**
-   * Creates the dummy data with a specific size.
-   * @param size the size of value
-   * @return the dummy data
-   */
-  private byte[] makeDummyData(int size) {
-    byte[] dummyData = new byte[size];
-    new Random().nextBytes(dummyData);
-    return dummyData;
-  }
-
-  /**
-   * Gets the split keys
-   */
-  private byte[][] getSplitKeys() {
-    byte[][] splitKeys = new byte[KEYS.length - 1][];
-    for (int i = 0; i < splitKeys.length; ++i) {
-      splitKeys[i] = new byte[] { KEYS[i + 1] };
-    }
-    return splitKeys;
-  }
-
-  private static ExecutorService createThreadPool(Configuration conf) {
-    int maxThreads = 10;
-    long keepAliveTime = 60;
-    final SynchronousQueue<Runnable> queue = new SynchronousQueue<>();
-    ThreadPoolExecutor pool = new ThreadPoolExecutor(1, maxThreads,
-        keepAliveTime, TimeUnit.SECONDS, queue,
-        Threads.newDaemonThreadFactory("MobFileCompactionChore"),
-        new RejectedExecutionHandler() {
-          @Override
-          public void rejectedExecution(Runnable r, ThreadPoolExecutor executor) {
-            try {
-              // waiting for a thread to pick up instead of throwing exceptions.
-              queue.put(r);
-            } catch (InterruptedException e) {
-              throw new RejectedExecutionException(e);
-            }
-          }
-        });
-    pool.allowCoreThreadTimeOut(true);
-    return pool;
-  }
-
-  private void assertRefFileNameEqual(String familyName) throws IOException {
-    Scan scan = new Scan();
-    scan.addFamily(Bytes.toBytes(familyName));
-    // Do not retrieve the mob data when scanning
-    scan.setAttribute(MobConstants.MOB_SCAN_RAW, Bytes.toBytes(Boolean.TRUE));
-    ResultScanner results = table.getScanner(scan);
-    Path mobFamilyPath = MobUtils.getMobFamilyPath(TEST_UTIL.getConfiguration(),
-        tableName, familyName);
-    List<Path> actualFilePaths = new ArrayList<>();
-    List<Path> expectFilePaths = new ArrayList<>();
-    for (Result res : results) {
-      for (Cell cell : res.listCells()) {
-        byte[] referenceValue = CellUtil.cloneValue(cell);
-        String fileName = Bytes.toString(referenceValue, Bytes.SIZEOF_INT,
-            referenceValue.length - Bytes.SIZEOF_INT);
-        Path targetPath = new Path(mobFamilyPath, fileName);
-        if(!actualFilePaths.contains(targetPath)) {
-          actualFilePaths.add(targetPath);
-        }
-      }
-    }
-    results.close();
-    if (fs.exists(mobFamilyPath)) {
-      FileStatus[] files = fs.listStatus(mobFamilyPath);
-      for (FileStatus file : files) {
-        if (!StoreFileInfo.isDelFile(file.getPath())) {
-          expectFilePaths.add(file.getPath());
-        }
-      }
-    }
-    Collections.sort(actualFilePaths);
-    Collections.sort(expectFilePaths);
-    assertEquals(expectFilePaths, actualFilePaths);
-  }
-
-  /**
-   * Resets the configuration.
-   */
-  private void resetConf() {
-    conf.setLong(MobConstants.MOB_COMPACTION_MERGEABLE_THRESHOLD,
-      MobConstants.DEFAULT_MOB_COMPACTION_MERGEABLE_THRESHOLD);
-    conf.setInt(MobConstants.MOB_COMPACTION_BATCH_SIZE,
-      MobConstants.DEFAULT_MOB_COMPACTION_BATCH_SIZE);
-  }
-
-  /**
-   * Verify mob partition policy compaction values.
-   */
-  private void verifyPolicyValues() throws Exception {
-    Get get = new Get(mobKey01);
-    Result result = table.get(get);
-    assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)),
-        Bytes.toBytes(mobValue0)));
-
-    get = new Get(mobKey02);
-    result = table.get(get);
-    assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)),
-        Bytes.toBytes(mobValue0)));
-
-    get = new Get(mobKey03);
-    result = table.get(get);
-    assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)),
-        Bytes.toBytes(mobValue0)));
-
-    get = new Get(mobKey04);
-    result = table.get(get);
-    assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)),
-        Bytes.toBytes(mobValue0)));
-
-    get = new Get(mobKey05);
-    result = table.get(get);
-    assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)),
-        Bytes.toBytes(mobValue0)));
-
-    get = new Get(mobKey06);
-    result = table.get(get);
-    assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)),
-        Bytes.toBytes(mobValue0)));
-
-    get = new Get(mobKey1);
-    result = table.get(get);
-    assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)),
-        Bytes.toBytes(mobValue1)));
-
-    get = new Get(mobKey2);
-    result = table.get(get);
-    assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)),
-        Bytes.toBytes(mobValue2)));
-
-    get = new Get(mobKey3);
-    result = table.get(get);
-    assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)),
-        Bytes.toBytes(mobValue3)));
-
-    get = new Get(mobKey4);
-    result = table.get(get);
-    assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)),
-        Bytes.toBytes(mobValue4)));
-
-    get = new Get(mobKey5);
-    result = table.get(get);
-    assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)),
-        Bytes.toBytes(mobValue5)));
-
-    get = new Get(mobKey6);
-    result = table.get(get);
-    assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)),
-        Bytes.toBytes(mobValue6)));
-
-    get = new Get(mobKey7);
-    result = table.get(get);
-    assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)),
-        Bytes.toBytes(mobValue7)));
-
-    get = new Get(mobKey8);
-    result = table.get(get);
-    assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)),
-        Bytes.toBytes(mobValue8)));
-  }
-
-  private void commonPolicyTestLogic (final String tableNameAsString,
-      final MobCompactPartitionPolicy pType, final boolean majorCompact,
-      final int expectedFileNumbers, final String[] expectedFileNames,
-      final boolean setupAndLoadData
-      ) throws Exception {
-    if (setupAndLoadData) {
-      setUpForPolicyTest(tableNameAsString, pType);
-
-      loadDataForPartitionPolicy(admin, bufMut, tableName);
-    } else {
-      alterForPolicyTest(pType);
-    }
-
-    if (majorCompact) {
-      admin.majorCompact(tableName, cfdb1.build().getName(), CompactType.MOB);
-    } else {
-      admin.compact(tableName, cfdb1.build().getName(), CompactType.MOB);
-    }
-
-    waitUntilMobCompactionFinished(tableName);
-
-    // Run cleaner to make sure that files in archive directory are cleaned up
-    TEST_UTIL.getMiniHBaseCluster().getMaster().getHFileCleaner().choreForTesting();
-
-    //check the number of files
-    Path mobDirPath = MobUtils.getMobFamilyPath(conf, tableName, family1);
-    FileStatus[] fileList = fs.listStatus(mobDirPath);
-
-    assertTrue(fileList.length == expectedFileNumbers);
-
-    // the file names are expected
-    ArrayList<String> fileNames = new ArrayList<>(expectedFileNumbers);
-    for (FileStatus file : fileList) {
-      fileNames.add(MobFileName.getDateFromName(file.getPath().getName()));
-    }
-    int index = 0;
-    for (String fileName : expectedFileNames) {
-      index = fileNames.indexOf(fileName);
-      assertTrue(index >= 0);
-      fileNames.remove(index);
-    }
-
-    // Check daily mob files are removed from the mobdir, and only weekly mob files are there.
-    // Also check that there is no data loss.
-
-    verifyPolicyValues();
-  }
- }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactionRequest.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactionRequest.java
deleted file mode 100644
index 2780153..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactionRequest.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.mob.compactions;
-
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HBaseClassTestRule;
-import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionPartition;
-import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionPartitionId;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.junit.Assert;
-import org.junit.ClassRule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category(SmallTests.class)
-public class TestPartitionedMobCompactionRequest {
-
-  @ClassRule
-  public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestPartitionedMobCompactionRequest.class);
-
-  @Test
-  public void testCompactedPartitionId() {
-    String startKey1 = "startKey1";
-    String startKey2 = "startKey2";
-    String date1 = "date1";
-    String date2 = "date2";
-    CompactionPartitionId partitionId1 = new CompactionPartitionId(startKey1, date1);
-    CompactionPartitionId partitionId2 = new CompactionPartitionId(startKey2, date2);
-    CompactionPartitionId partitionId3 = new CompactionPartitionId(startKey1, date2);
-
-    Assert.assertTrue(partitionId1.equals(partitionId1));
-    Assert.assertFalse(partitionId1.equals(partitionId2));
-    Assert.assertFalse(partitionId1.equals(partitionId3));
-    Assert.assertFalse(partitionId2.equals(partitionId3));
-
-    Assert.assertEquals(startKey1, partitionId1.getStartKey());
-    Assert.assertEquals(date1, partitionId1.getDate());
-  }
-
-  @Test
-  public void testCompactedPartition() {
-    CompactionPartitionId partitionId = new CompactionPartitionId("startKey1", "date1");
-    CompactionPartition partition = new CompactionPartition(partitionId);
-    FileStatus file = new FileStatus(1, false, 1, 1024, 1, new Path("/test"));
-    partition.addFile(file);
-    Assert.assertEquals(file, partition.listFiles().get(0));
-  }
-}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactor.java
deleted file mode 100644
index 73ee965..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactor.java
+++ /dev/null
@@ -1,961 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.mob.compactions;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-import java.text.ParseException;
-import java.util.ArrayList;
-import java.util.Calendar;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.RejectedExecutionException;
-import java.util.concurrent.RejectedExecutionHandler;
-import java.util.concurrent.SynchronousQueue;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.CellComparatorImpl;
-import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.HBaseClassTestRule;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.KeyValue.Type;
-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.MobCompactPartitionPolicy;
-import org.apache.hadoop.hbase.io.hfile.CacheConfig;
-import org.apache.hadoop.hbase.io.hfile.HFileContext;
-import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
-import org.apache.hadoop.hbase.mob.MobConstants;
-import org.apache.hadoop.hbase.mob.MobFileName;
-import org.apache.hadoop.hbase.mob.MobUtils;
-import org.apache.hadoop.hbase.mob.compactions.MobCompactionRequest.CompactionType;
-import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionDelPartition;
-import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionPartition;
-import org.apache.hadoop.hbase.regionserver.BloomType;
-import org.apache.hadoop.hbase.regionserver.HStore;
-import org.apache.hadoop.hbase.regionserver.HStoreFile;
-import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
-import org.apache.hadoop.hbase.regionserver.ScanInfo;
-import org.apache.hadoop.hbase.regionserver.ScanType;
-import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
-import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
-import org.apache.hadoop.hbase.regionserver.StoreScanner;
-import org.apache.hadoop.hbase.testclassification.LargeTests;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.hadoop.hbase.util.Threads;
-import org.apache.hadoop.hdfs.DistributedFileSystem;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.ClassRule;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.rules.TestName;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-@Category(LargeTests.class)
-public class TestPartitionedMobCompactor {
-
-  @ClassRule
-  public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestPartitionedMobCompactor.class);
-
-  private static final Logger LOG = LoggerFactory.getLogger(TestPartitionedMobCompactor.class);
-  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-  private final static String family = "family";
-  private final static String qf = "qf";
-  private final long DAY_IN_MS = 1000 * 60 * 60 * 24;
-  private static byte[] KEYS = Bytes.toBytes("012");
-  private ColumnFamilyDescriptorBuilder cfdb =
-    ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(family));
-  private Configuration conf = TEST_UTIL.getConfiguration();
-  private CacheConfig cacheConf = new CacheConfig(conf);
-  private FileSystem fs;
-  private List<FileStatus> mobFiles = new ArrayList<>();
-  private List<Path> delFiles = new ArrayList<>();
-  private List<FileStatus> allFiles = new ArrayList<>();
-  private Path basePath;
-  private String mobSuffix;
-  private String delSuffix;
-  private static ExecutorService pool;
-
-  @Rule
-  public TestName name = new TestName();
-
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    TEST_UTIL.getConfiguration().setInt("hfile.format.version", 3);
-    // Inject our customized DistributedFileSystem
-    TEST_UTIL.getConfiguration().setClass("fs.hdfs.impl", FaultyDistributedFileSystem.class,
-        DistributedFileSystem.class);
-    TEST_UTIL.startMiniCluster(1);
-    pool = createThreadPool();
-  }
-
-  @AfterClass
-  public static void tearDownAfterClass() throws Exception {
-    pool.shutdown();
-    TEST_UTIL.shutdownMiniCluster();
-  }
-
-  private void init(String tableName) throws Exception {
-    fs = FileSystem.get(conf);
-    Path testDir = FSUtils.getRootDir(conf);
-    Path mobTestDir = new Path(testDir, MobConstants.MOB_DIR_NAME);
-    basePath = new Path(new Path(mobTestDir, tableName), family);
-    mobSuffix = TEST_UTIL.getRandomUUID().toString().replaceAll("-", "");
-    delSuffix = TEST_UTIL.getRandomUUID().toString().replaceAll("-", "") + "_del";
-    allFiles.clear();
-    mobFiles.clear();
-    delFiles.clear();
-  }
-
-  @Test
-  public void testCompactionSelectAllFilesWeeklyPolicy() throws Exception {
-    String tableName = "testCompactionSelectAllFilesWeeklyPolicy";
-    testCompactionAtMergeSize(tableName, MobConstants.DEFAULT_MOB_COMPACTION_MERGEABLE_THRESHOLD,
-        CompactionType.ALL_FILES, false, false, new Date(), MobCompactPartitionPolicy.WEEKLY, 1);
-  }
-
-  @Test
-  public void testCompactionSelectPartFilesWeeklyPolicy() throws Exception {
-    String tableName = "testCompactionSelectPartFilesWeeklyPolicy";
-    testCompactionAtMergeSize(tableName, 4000, CompactionType.PART_FILES, false, false,
-        new Date(), MobCompactPartitionPolicy.WEEKLY, 1);
-  }
-
-  @Test
-  public void testCompactionSelectPartFilesWeeklyPolicyWithPastWeek() throws Exception {
-    String tableName = "testCompactionSelectPartFilesWeeklyPolicyWithPastWeek";
-    Date dateLastWeek = new Date(System.currentTimeMillis() - (7 * DAY_IN_MS));
-    testCompactionAtMergeSize(tableName, 700, CompactionType.PART_FILES, false, false, dateLastWeek,
-        MobCompactPartitionPolicy.WEEKLY, 7);
-  }
-
-  @Test
-  public void testCompactionSelectAllFilesWeeklyPolicyWithPastWeek() throws Exception {
-    String tableName = "testCompactionSelectAllFilesWeeklyPolicyWithPastWeek";
-    Date dateLastWeek = new Date(System.currentTimeMillis() - (7 * DAY_IN_MS));
-    testCompactionAtMergeSize(tableName, 3000, CompactionType.ALL_FILES,
-        false, false, dateLastWeek, MobCompactPartitionPolicy.WEEKLY, 7);
-  }
-
-  @Test
-  public void testCompactionSelectAllFilesMonthlyPolicy() throws Exception {
-    String tableName = "testCompactionSelectAllFilesMonthlyPolicy";
-    Date dateLastWeek = new Date(System.currentTimeMillis() - (7 * DAY_IN_MS));
-    testCompactionAtMergeSize(tableName, MobConstants.DEFAULT_MOB_COMPACTION_MERGEABLE_THRESHOLD,
-        CompactionType.ALL_FILES, false, false, dateLastWeek,
-        MobCompactPartitionPolicy.MONTHLY, 7);
-  }
-
-  @Test
-  public void testCompactionSelectNoFilesWithinCurrentWeekMonthlyPolicy() throws Exception {
-    String tableName = "testCompactionSelectNoFilesWithinCurrentWeekMonthlyPolicy";
-    testCompactionAtMergeSize(tableName, MobConstants.DEFAULT_MOB_COMPACTION_MERGEABLE_THRESHOLD,
-        CompactionType.PART_FILES, false, false, new Date(), MobCompactPartitionPolicy.MONTHLY, 1);
-  }
-
-  @Test
-  public void testCompactionSelectPartFilesMonthlyPolicy() throws Exception {
-    String tableName = "testCompactionSelectPartFilesMonthlyPolicy";
-    testCompactionAtMergeSize(tableName, 4000, CompactionType.PART_FILES, false, false,
-        new Date(), MobCompactPartitionPolicy.MONTHLY, 1);
-  }
-
-  @Test
-  public void testCompactionSelectPartFilesMonthlyPolicyWithPastWeek() throws Exception {
-    String tableName = "testCompactionSelectPartFilesMonthlyPolicyWithPastWeek";
-    Date dateLastWeek = new Date(System.currentTimeMillis() - (7 * DAY_IN_MS));
-    Calendar calendar =  Calendar.getInstance();
-    Date firstDayOfCurrentMonth = MobUtils.getFirstDayOfMonth(calendar, new Date());
-    CompactionType type = CompactionType.PART_FILES;
-    long mergeSizeMultiFactor = 7;
-
-
-    // The dateLastWeek may not really be last week, suppose that it runs at 2/1/2017, it is going
-    // to be last month and the monthly policy is going to be applied here.
-    if (dateLastWeek.before(firstDayOfCurrentMonth)) {
-      type = CompactionType.ALL_FILES;
-      mergeSizeMultiFactor *= 4;
-    }
-
-    testCompactionAtMergeSize(tableName, 700, type, false, false, dateLastWeek,
-        MobCompactPartitionPolicy.MONTHLY, mergeSizeMultiFactor);
-  }
-
-  @Test
-  public void testCompactionSelectAllFilesMonthlyPolicyWithPastWeek() throws Exception {
-    String tableName = "testCompactionSelectAllFilesMonthlyPolicyWithPastWeek";
-    Date dateLastWeek = new Date(System.currentTimeMillis() - (7 * DAY_IN_MS));
-
-    testCompactionAtMergeSize(tableName, 3000, CompactionType.ALL_FILES,
-        false, false, dateLastWeek, MobCompactPartitionPolicy.MONTHLY, 7);
-  }
-
-  @Test
-  public void testCompactionSelectPartFilesMonthlyPolicyWithPastMonth() throws Exception {
-    String tableName = "testCompactionSelectPartFilesMonthlyPolicyWithPastMonth";
-
-    // back 5 weeks, it is going to be a past month
-    Date dateLastMonth = new Date(System.currentTimeMillis() - (7 * 5 * DAY_IN_MS));
-    testCompactionAtMergeSize(tableName, 200, CompactionType.PART_FILES, false, false, dateLastMonth,
-        MobCompactPartitionPolicy.MONTHLY, 28);
-  }
-
-  @Test
-  public void testCompactionSelectAllFilesMonthlyPolicyWithPastMonth() throws Exception {
-    String tableName = "testCompactionSelectAllFilesMonthlyPolicyWithPastMonth";
-
-    // back 5 weeks, it is going to be a past month
-    Date dateLastMonth = new Date(System.currentTimeMillis() - (7 * 5 * DAY_IN_MS));
-    testCompactionAtMergeSize(tableName, 750, CompactionType.ALL_FILES,
-        false, false, dateLastMonth, MobCompactPartitionPolicy.MONTHLY, 28);
-  }
-
-  @Test
-  public void testCompactionSelectWithAllFiles() throws Exception {
-    String tableName = "testCompactionSelectWithAllFiles";
-    // If there is only 1 file, it will not be compacted with _del files, so
-    // It wont be CompactionType.ALL_FILES in this case, do not create with _del files.
-    testCompactionAtMergeSize(tableName, MobConstants.DEFAULT_MOB_COMPACTION_MERGEABLE_THRESHOLD,
-        CompactionType.ALL_FILES, false, false);
-  }
-
-  @Test
-  public void testCompactionSelectWithPartFiles() throws Exception {
-    String tableName = "testCompactionSelectWithPartFiles";
-    testCompactionAtMergeSize(tableName, 4000, CompactionType.PART_FILES, false);
-  }
-
-  @Test
-  public void testCompactionSelectWithForceAllFiles() throws Exception {
-    String tableName = "testCompactionSelectWithForceAllFiles";
-    testCompactionAtMergeSize(tableName, Long.MAX_VALUE, CompactionType.ALL_FILES, true);
-  }
-
-  private void testCompactionAtMergeSize(final String tableName,
-      final long mergeSize, final CompactionType type, final boolean isForceAllFiles)
-      throws Exception {
-    testCompactionAtMergeSize(tableName, mergeSize, type, isForceAllFiles, true);
-  }
-
-  private void testCompactionAtMergeSize(final String tableName,
-      final long mergeSize, final CompactionType type, final boolean isForceAllFiles,
-      final boolean createDelFiles)
-      throws Exception {
-    Date date = new Date();
-    testCompactionAtMergeSize(tableName, mergeSize, type, isForceAllFiles, createDelFiles, date);
-  }
-
-  private void testCompactionAtMergeSize(final String tableName,
-      final long mergeSize, final CompactionType type, final boolean isForceAllFiles,
-      final boolean createDelFiles, final Date date)
-      throws Exception {
-    testCompactionAtMergeSize(tableName, mergeSize, type, isForceAllFiles, createDelFiles, date,
-        MobCompactPartitionPolicy.DAILY, 1);
-  }
-
-  private void testCompactionAtMergeSize(final String tableName,
-      final long mergeSize, final CompactionType type, final boolean isForceAllFiles,
-      final boolean createDelFiles, final Date date, final MobCompactPartitionPolicy policy,
-      final long mergeSizeMultiFactor)
-      throws Exception {
-    resetConf();
-    init(tableName);
-    int count = 10;
-    // create 10 mob files.
-    createStoreFiles(basePath, family, qf, count, Type.Put, date);
-
-    if (createDelFiles) {
-      // create 10 del files
-      createStoreFiles(basePath, family, qf, count, Type.Delete, date);
-    }
-
-    Calendar calendar =  Calendar.getInstance();
-    Date firstDayOfCurrentWeek = MobUtils.getFirstDayOfWeek(calendar, new Date());
-
-    listFiles();
-    List<String> expectedStartKeys = new ArrayList<>();
-    for(FileStatus file : mobFiles) {
-      if(file.getLen() < mergeSize * mergeSizeMultiFactor) {
-        String fileName = file.getPath().getName();
-        String startKey = fileName.substring(0, 32);
-
-        // If the policy is monthly and files are in current week, they will be skipped
-        // in minor compcation.
-        boolean skipCompaction = false;
-        if (policy == MobCompactPartitionPolicy.MONTHLY) {
-          String fileDateStr = MobFileName.getDateFromName(fileName);
-          Date fileDate;
-          try {
-            fileDate = MobUtils.parseDate(fileDateStr);
-          } catch (ParseException e)  {
-            LOG.warn("Failed to parse date " + fileDateStr, e);
-            fileDate = new Date();
-          }
-          if (!fileDate.before(firstDayOfCurrentWeek)) {
-            skipCompaction = true;
-          }
-        }
-
-        // If it is not an major mob compaction and del files are there,
-        // these mob files wont be compacted.
-        if (isForceAllFiles || (!createDelFiles && !skipCompaction)) {
-          expectedStartKeys.add(startKey);
-        }
-      }
-    }
-
-    // Set the policy
-    this.cfdb.setMobCompactPartitionPolicy(policy);
-    // set the mob compaction mergeable threshold
-    conf.setLong(MobConstants.MOB_COMPACTION_MERGEABLE_THRESHOLD, mergeSize);
-    testSelectFiles(tableName, type, isForceAllFiles, expectedStartKeys);
-    // go back to the default daily policy
-    this.cfdb.setMobCompactPartitionPolicy(MobCompactPartitionPolicy.DAILY);
-  }
-
-  @Test
-  public void testCompactDelFilesWithDefaultBatchSize() throws Exception {
-    testCompactDelFilesAtBatchSize(name.getMethodName(), MobConstants.DEFAULT_MOB_COMPACTION_BATCH_SIZE,
-        MobConstants.DEFAULT_MOB_DELFILE_MAX_COUNT);
-  }
-
-  @Test
-  public void testCompactDelFilesWithSmallBatchSize() throws Exception {
-    testCompactDelFilesAtBatchSize(name.getMethodName(), 4, MobConstants.DEFAULT_MOB_DELFILE_MAX_COUNT);
-  }
-
-  @Test
-  public void testCompactDelFilesChangeMaxDelFileCount() throws Exception {
-    testCompactDelFilesAtBatchSize(name.getMethodName(), 4, 2);
-  }
-
-  @Test
-  public void testCompactFilesWithDstDirFull() throws Exception {
-    String tableName = name.getMethodName();
-    fs = FileSystem.get(conf);
-    FaultyDistributedFileSystem faultyFs = (FaultyDistributedFileSystem)fs;
-    Path testDir = FSUtils.getRootDir(conf);
-    Path mobTestDir = new Path(testDir, MobConstants.MOB_DIR_NAME);
-    basePath = new Path(new Path(mobTestDir, tableName), family);
-
-    try {
-      int count = 2;
-      // create 2 mob files.
-      createStoreFiles(basePath, family, qf, count, Type.Put, true, new Date());
-      listFiles();
-
-      TableName tName = TableName.valueOf(tableName);
-      MobCompactor compactor = new PartitionedMobCompactor(conf, faultyFs, tName,
-        cfdb.build(), pool);
-      faultyFs.setThrowException(true);
-      try {
-        compactor.compact(allFiles, true);
-      } catch (IOException e) {
-        System.out.println("Expected exception, ignore");
-      }
-
-      // Verify that all the files in tmp directory are cleaned up
-      Path tempPath = new Path(MobUtils.getMobHome(conf), MobConstants.TEMP_DIR_NAME);
-      FileStatus[] ls = faultyFs.listStatus(tempPath);
-
-      // Only .bulkload under this directory
-      assertTrue(ls.length == 1);
-      assertTrue(MobConstants.BULKLOAD_DIR_NAME.equalsIgnoreCase(ls[0].getPath().getName()));
-
-      Path bulkloadPath = new Path(tempPath, new Path(MobConstants.BULKLOAD_DIR_NAME, new Path(
-          tName.getNamespaceAsString(), tName.getQualifierAsString())));
-
-      // Nothing in bulkLoad directory
-      FileStatus[] lsBulkload = faultyFs.listStatus(bulkloadPath);
-      assertTrue(lsBulkload.length == 0);
-
-    } finally {
-      faultyFs.setThrowException(false);
-    }
-  }
-
-  /**
-   * Create mulitple partition files
-   */
-  private void createMobFile(Path basePath) throws IOException {
-    HFileContext meta = new HFileContextBuilder().withBlockSize(8 * 1024).build();
-    MobFileName mobFileName = null;
-    int ii = 0;
-    Date today = new Date();
-    for (byte k0 : KEYS) {
-      byte[] startRow = Bytes.toBytes(ii++);
-
-      mobFileName = MobFileName.create(startRow, MobUtils.formatDate(today), mobSuffix);
-
-      StoreFileWriter mobFileWriter =
-          new StoreFileWriter.Builder(conf, cacheConf, fs).withFileContext(meta)
-              .withFilePath(new Path(basePath, mobFileName.getFileName())).build();
-
-      long now = System.currentTimeMillis();
-      try {
-        for (int i = 0; i < 10; i++) {
-          byte[] key = Bytes.add(Bytes.toBytes(k0), Bytes.toBytes(i));
-          byte[] dummyData = new byte[5000];
-          new Random().nextBytes(dummyData);
-          mobFileWriter.append(
-              new KeyValue(key, Bytes.toBytes(family), Bytes.toBytes(qf), now, Type.Put, dummyData));
-        }
-      } finally {
-        mobFileWriter.close();
-      }
-    }
-  }
-
-  /**
-   * Create mulitple partition delete files
-   */
-  private void createMobDelFile(Path basePath, int startKey) throws IOException {
-    HFileContext meta = new HFileContextBuilder().withBlockSize(8 * 1024).build();
-    MobFileName mobFileName = null;
-    Date today = new Date();
-
-    byte[] startRow = Bytes.toBytes(startKey);
-
-    mobFileName = MobFileName.create(startRow, MobUtils.formatDate(today), delSuffix);
-
-    StoreFileWriter mobFileWriter =
-        new StoreFileWriter.Builder(conf, cacheConf, fs).withFileContext(meta)
-            .withFilePath(new Path(basePath, mobFileName.getFileName())).build();
-
-    long now = System.currentTimeMillis();
-    try {
-      byte[] key = Bytes.add(Bytes.toBytes(KEYS[startKey]), Bytes.toBytes(0));
-      byte[] dummyData = new byte[5000];
-      new Random().nextBytes(dummyData);
-      mobFileWriter.append(
-          new KeyValue(key, Bytes.toBytes(family), Bytes.toBytes(qf), now, Type.Delete, dummyData));
-      key = Bytes.add(Bytes.toBytes(KEYS[startKey]), Bytes.toBytes(2));
-      mobFileWriter.append(
-          new KeyValue(key, Bytes.toBytes(family), Bytes.toBytes(qf), now, Type.Delete, dummyData));
-      key = Bytes.add(Bytes.toBytes(KEYS[startKey]), Bytes.toBytes(4));
-      mobFileWriter.append(
-          new KeyValue(key, Bytes.toBytes(family), Bytes.toBytes(qf), now, Type.Delete, dummyData));
-
-    } finally {
-      mobFileWriter.close();
-    }
-  }
-
-  @Test
-  public void testCompactFilesWithoutDelFile() throws Exception {
-    String tableName = "testCompactFilesWithoutDelFile";
-    resetConf();
-    init(tableName);
-
-    createMobFile(basePath);
-
-    listFiles();
-
-    PartitionedMobCompactor compactor = new PartitionedMobCompactor(conf, fs,
-        TableName.valueOf(tableName), cfdb.build(), pool) {
-      @Override
-      public List<Path> compact(List<FileStatus> files, boolean isForceAllFiles)
-          throws IOException {
-        if (files == null || files.isEmpty()) {
-          return null;
-        }
-
-        PartitionedMobCompactionRequest request = select(files, isForceAllFiles);
-
-        // Make sure that there is no del Partitions
-        assertTrue(request.getDelPartitions().size() == 0);
-
-        // Make sure that when there is no startKey/endKey for partition.
-        for (CompactionPartition p : request.getCompactionPartitions()) {
-          assertTrue(p.getStartKey() == null);
-          assertTrue(p.getEndKey() == null);
-        }
-        return null;
-      }
-    };
-
-    compactor.compact(allFiles, true);
-  }
-
-  static class MyPartitionedMobCompactor extends PartitionedMobCompactor {
-    int delPartitionSize = 0;
-    int PartitionsIncludeDelFiles = 0;
-    CacheConfig cacheConfig = null;
-
-    MyPartitionedMobCompactor(Configuration conf, FileSystem fs, TableName tableName,
-        ColumnFamilyDescriptor column, ExecutorService pool, final int delPartitionSize,
-        final CacheConfig cacheConf, final int PartitionsIncludeDelFiles)
-        throws IOException {
-      super(conf, fs, tableName, column, pool);
-      this.delPartitionSize = delPartitionSize;
-      this.cacheConfig = cacheConf;
-      this.PartitionsIncludeDelFiles = PartitionsIncludeDelFiles;
-    }
-
-    @Override public List<Path> compact(List<FileStatus> files, boolean isForceAllFiles)
-        throws IOException {
-      if (files == null || files.isEmpty()) {
-        return null;
-      }
-      PartitionedMobCompactionRequest request = select(files, isForceAllFiles);
-
-      assertTrue(request.getDelPartitions().size() == delPartitionSize);
-      if (request.getDelPartitions().size() > 0) {
-        for (CompactionPartition p : request.getCompactionPartitions()) {
-          assertTrue(p.getStartKey() != null);
-          assertTrue(p.getEndKey() != null);
-        }
-      }
-
-      try {
-        for (CompactionDelPartition delPartition : request.getDelPartitions()) {
-          for (Path newDelPath : delPartition.listDelFiles()) {
-            HStoreFile sf =
-                new HStoreFile(fs, newDelPath, conf, this.cacheConfig, BloomType.NONE, true);
-            // pre-create reader of a del file to avoid race condition when opening the reader in
-            // each partition.
-            sf.initReader();
-            delPartition.addStoreFile(sf);
-          }
-        }
-
-        // Make sure that CompactionDelPartitions does not overlap
-        CompactionDelPartition prevDelP = null;
-        for (CompactionDelPartition delP : request.getDelPartitions()) {
-          assertTrue(
-              Bytes.compareTo(delP.getId().getStartKey(), delP.getId().getEndKey()) <= 0);
-
-          if (prevDelP != null) {
-            assertTrue(
-                Bytes.compareTo(prevDelP.getId().getEndKey(), delP.getId().getStartKey()) < 0);
-          }
-        }
-
-        int affectedPartitions = 0;
-
-        // Make sure that only del files within key range for a partition is included in compaction.
-        // compact the mob files by partitions in parallel.
-        for (CompactionPartition partition : request.getCompactionPartitions()) {
-          List<HStoreFile> delFiles = getListOfDelFilesForPartition(partition, request.getDelPartitions());
-          if (!request.getDelPartitions().isEmpty()) {
-            if (!((Bytes.compareTo(request.getDelPartitions().get(0).getId().getStartKey(),
-                partition.getEndKey()) > 0) || (Bytes.compareTo(
-                request.getDelPartitions().get(request.getDelPartitions().size() - 1).getId()
-                    .getEndKey(), partition.getStartKey()) < 0))) {
-
-              if (delFiles.size() > 0) {
-                assertTrue(delFiles.size() == 1);
-                affectedPartitions += delFiles.size();
-                assertTrue(Bytes.compareTo(partition.getStartKey(),
-                  CellUtil.cloneRow(delFiles.get(0).getLastKey().get())) <= 0);
-                assertTrue(Bytes.compareTo(partition.getEndKey(),
-                  CellUtil.cloneRow(delFiles.get(delFiles.size() - 1).getFirstKey().get())) >= 0);
-              }
-            }
-          }
-        }
-        // The del file is only included in one partition
-        assertTrue(affectedPartitions == PartitionsIncludeDelFiles);
-      } finally {
-        for (CompactionDelPartition delPartition : request.getDelPartitions()) {
-          for (HStoreFile storeFile : delPartition.getStoreFiles()) {
-            try {
-              storeFile.closeStoreFile(true);
-            } catch (IOException e) {
-              LOG.warn("Failed to close the reader on store file " + storeFile.getPath(), e);
-            }
-          }
-        }
-      }
-
-      return null;
-    }
-  }
-
-  @Test
-  public void testCompactFilesWithOneDelFile() throws Exception {
-    String tableName = "testCompactFilesWithOneDelFile";
-    resetConf();
-    init(tableName);
-
-    // Create only del file.
-    createMobFile(basePath);
-    createMobDelFile(basePath, 2);
-
-    listFiles();
-
-    MyPartitionedMobCompactor compactor = new MyPartitionedMobCompactor(conf, fs,
-        TableName.valueOf(tableName), cfdb.build(), pool, 1, cacheConf, 1);
-
-    compactor.compact(allFiles, true);
-  }
-
-  @Test
-  public void testCompactFilesWithMultiDelFiles() throws Exception {
-    String tableName = "testCompactFilesWithMultiDelFiles";
-    resetConf();
-    init(tableName);
-
-    // Create only del file.
-    createMobFile(basePath);
-    createMobDelFile(basePath, 0);
-    createMobDelFile(basePath, 1);
-    createMobDelFile(basePath, 2);
-
-    listFiles();
-
-    MyPartitionedMobCompactor compactor = new MyPartitionedMobCompactor(conf, fs,
-        TableName.valueOf(tableName), cfdb.build(), pool, 3, cacheConf, 3);
-    compactor.compact(allFiles, true);
-  }
-
-  private void testCompactDelFilesAtBatchSize(String tableName, int batchSize,
-      int delfileMaxCount)  throws Exception {
-    resetConf();
-    init(tableName);
-    // create 20 mob files.
-    createStoreFiles(basePath, family, qf, 20, Type.Put, new Date());
-    // create 13 del files
-    createStoreFiles(basePath, family, qf, 13, Type.Delete, new Date());
-    listFiles();
-
-    // set the max del file count
-    conf.setInt(MobConstants.MOB_DELFILE_MAX_COUNT, delfileMaxCount);
-    // set the mob compaction batch size
-    conf.setInt(MobConstants.MOB_COMPACTION_BATCH_SIZE, batchSize);
-    testCompactDelFiles(tableName, 1, 13, false);
-  }
-
-  /**
-   * Tests the selectFiles
-   * @param tableName the table name
-   * @param type the expected compaction type
-   * @param isForceAllFiles whether all the mob files are selected
-   * @param expected the expected start keys
-   */
-  private void testSelectFiles(String tableName, final CompactionType type,
-    final boolean isForceAllFiles, final List<String> expected) throws IOException {
-    PartitionedMobCompactor compactor = new PartitionedMobCompactor(conf, fs,
-      TableName.valueOf(tableName), cfdb.build(), pool) {
-      @Override
-      public List<Path> compact(List<FileStatus> files, boolean isForceAllFiles)
-        throws IOException {
-        if (files == null || files.isEmpty()) {
-          return null;
-        }
-        PartitionedMobCompactionRequest request = select(files, isForceAllFiles);
-
-        // Make sure that when there is no del files, there will be no startKey/endKey for partition.
-        if (request.getDelPartitions().size() == 0) {
-          for (CompactionPartition p : request.getCompactionPartitions()) {
-            assertTrue(p.getStartKey() == null);
-            assertTrue(p.getEndKey() == null);
-          }
-        }
-
-        // Make sure that CompactionDelPartitions does not overlap
-        CompactionDelPartition prevDelP = null;
-        for (CompactionDelPartition delP : request.getDelPartitions()) {
-          assertTrue(Bytes.compareTo(delP.getId().getStartKey(),
-              delP.getId().getEndKey()) <= 0);
-
-          if (prevDelP != null) {
-            assertTrue(Bytes.compareTo(prevDelP.getId().getEndKey(),
-                delP.getId().getStartKey()) < 0);
-          }
-        }
-
-        // Make sure that only del files within key range for a partition is included in compaction.
-        // compact the mob files by partitions in parallel.
-        for (CompactionPartition partition : request.getCompactionPartitions()) {
-          List<HStoreFile> delFiles = getListOfDelFilesForPartition(partition, request.getDelPartitions());
-          if (!request.getDelPartitions().isEmpty()) {
-            if (!((Bytes.compareTo(request.getDelPartitions().get(0).getId().getStartKey(),
-                partition.getEndKey()) > 0) || (Bytes.compareTo(
-                request.getDelPartitions().get(request.getDelPartitions().size() - 1).getId()
-                    .getEndKey(), partition.getStartKey()) < 0))) {
-              if (delFiles.size() > 0) {
-                assertTrue(Bytes.compareTo(partition.getStartKey(),
-                  delFiles.get(0).getFirstKey().get().getRowArray()) >= 0);
-                assertTrue(Bytes.compareTo(partition.getEndKey(),
-                  delFiles.get(delFiles.size() - 1).getLastKey().get().getRowArray()) <= 0);
-              }
-            }
-          }
-        }
-
-        // assert the compaction type
-        assertEquals(type, request.type);
-        // assert get the right partitions
-        compareCompactedPartitions(expected, request.compactionPartitions);
-        // assert get the right del files
-        compareDelFiles(request.getDelPartitions());
-        return null;
-      }
-    };
-    compactor.compact(allFiles, isForceAllFiles);
-  }
-
-  /**
-   * Tests the compacteDelFile
-   * @param tableName the table name
-   * @param expectedFileCount the expected file count
-   * @param expectedCellCount the expected cell count
-   * @param isForceAllFiles whether all the mob files are selected
-   */
-  private void testCompactDelFiles(String tableName, final int expectedFileCount,
-      final int expectedCellCount, boolean isForceAllFiles) throws IOException {
-    PartitionedMobCompactor compactor = new PartitionedMobCompactor(conf, fs,
-      TableName.valueOf(tableName), cfdb.build(), pool) {
-      @Override
-      protected List<Path> performCompaction(PartitionedMobCompactionRequest request)
-          throws IOException {
-        List<Path> delFilePaths = new ArrayList<>();
-        for (CompactionDelPartition delPartition: request.getDelPartitions()) {
-          for (Path p : delPartition.listDelFiles()) {
-            delFilePaths.add(p);
-          }
-        }
-        List<Path> newDelPaths = compactDelFiles(request, delFilePaths);
-        // assert the del files are merged.
-        assertEquals(expectedFileCount, newDelPaths.size());
-        assertEquals(expectedCellCount, countDelCellsInDelFiles(newDelPaths));
-        return null;
-      }
-    };
-    compactor.compact(allFiles, isForceAllFiles);
-  }
-
-  /**
-   * Lists the files in the path
-   */
-  private void listFiles() throws IOException {
-    for (FileStatus file : fs.listStatus(basePath)) {
-      allFiles.add(file);
-      if (file.getPath().getName().endsWith("_del")) {
-        delFiles.add(file.getPath());
-      } else {
-        mobFiles.add(file);
-      }
-    }
-  }
-
-  /**
-   * Compares the compacted partitions.
-   * @param partitions the collection of CompactedPartitions
-   */
-  private void compareCompactedPartitions(List<String> expected,
-      Collection<CompactionPartition> partitions) {
-    List<String> actualKeys = new ArrayList<>();
-    for (CompactionPartition partition : partitions) {
-      actualKeys.add(partition.getPartitionId().getStartKey());
-    }
-    Collections.sort(expected);
-    Collections.sort(actualKeys);
-    assertEquals(expected.size(), actualKeys.size());
-    for (int i = 0; i < expected.size(); i++) {
-      assertEquals(expected.get(i), actualKeys.get(i));
-    }
-  }
-
-  /**
-   * Compares the del files.
-   * @param delPartitions all del partitions
-   */
-  private void compareDelFiles(List<CompactionDelPartition> delPartitions) {
-    Map<Path, Path> delMap = new HashMap<>();
-    for (CompactionDelPartition delPartition : delPartitions) {
-      for (Path f : delPartition.listDelFiles()) {
-        delMap.put(f, f);
-      }
-    }
-    for (Path f : delFiles) {
-      assertTrue(delMap.containsKey(f));
-    }
-  }
-
-  /**
-   * Creates store files.
-   * @param basePath the path to create file
-   * @param family the column family name
-   * @param qualifier the column qualifier assigned to data values
-   * @param count the store file number
-   * @param type the row key type
-   * @param date the latest timestamp when an instance of MobFileName is created
-   */
-  private void createStoreFiles(Path basePath, String family, String qualifier, int count,
-      Type type, final Date date) throws IOException {
-    createStoreFiles(basePath, family, qualifier, count, type, false, date);
-  }
-
-  private void createStoreFiles(Path basePath, String family, String qualifier, int count,
-      Type type, boolean sameStartKey, final Date date) throws IOException {
-    HFileContext meta = new HFileContextBuilder().withBlockSize(8 * 1024).build();
-    String startKey = "row_";
-    MobFileName mobFileName = null;
-    for (int i = 0; i < count; i++) {
-      byte[] startRow;
-      if (sameStartKey) {
-        // When creating multiple files under one partition, suffix needs to be different.
-        startRow = Bytes.toBytes(startKey);
-        mobSuffix = TEST_UTIL.getRandomUUID().toString().replaceAll("-", "");
-        delSuffix = TEST_UTIL.getRandomUUID().toString().replaceAll("-", "") + "_del";
-      } else {
-        startRow = Bytes.toBytes(startKey + i);
-      }
-      if(type.equals(Type.Delete)) {
-        mobFileName = MobFileName.create(startRow, MobUtils.formatDate(date), delSuffix);
-      }
-      if(type.equals(Type.Put)){
-        mobFileName = MobFileName.create(startRow, MobUtils.formatDate(date), mobSuffix);
-      }
-      StoreFileWriter mobFileWriter = new StoreFileWriter.Builder(conf, cacheConf, fs)
-      .withFileContext(meta).withFilePath(new Path(basePath, mobFileName.getFileName())).build();
-      writeStoreFile(mobFileWriter, startRow, Bytes.toBytes(family), Bytes.toBytes(qualifier),
-          type, (i+1)*1000);
-    }
-  }
-
-  /**
-   * Writes data to store file.
-   * @param writer the store file writer
-   * @param row the row key
-   * @param family the family name
-   * @param qualifier the column qualifier
-   * @param type the key type
-   * @param size the size of value
-   */
-  private static void writeStoreFile(final StoreFileWriter writer, byte[]row, byte[] family,
-      byte[] qualifier, Type type, int size) throws IOException {
-    long now = System.currentTimeMillis();
-    try {
-      byte[] dummyData = new byte[size];
-      new Random().nextBytes(dummyData);
-      writer.append(new KeyValue(row, family, qualifier, now, type, dummyData));
-    } finally {
-      writer.close();
-    }
-  }
-
-  /**
-   * Gets the number of del cell in the del files
-   * @param paths the del file paths
-   * @return the cell size
-   */
-  private int countDelCellsInDelFiles(List<Path> paths) throws IOException {
-    List<HStoreFile> sfs = new ArrayList<>();
-    int size = 0;
-    for (Path path : paths) {
-      HStoreFile sf = new HStoreFile(fs, path, conf, cacheConf, BloomType.NONE, true);
-      sfs.add(sf);
-    }
-    List<KeyValueScanner> scanners = new ArrayList<>(StoreFileScanner.getScannersForStoreFiles(sfs,
-      false, true, false, false, HConstants.LATEST_TIMESTAMP));
-    long timeToPurgeDeletes = Math.max(conf.getLong("hbase.hstore.time.to.purge.deletes", 0), 0);
-    long ttl = HStore.determineTTLFromFamily(cfdb.build());
-    ScanInfo scanInfo = new ScanInfo(conf, cfdb.build(), ttl, timeToPurgeDeletes,
-      CellComparatorImpl.COMPARATOR);
-    StoreScanner scanner = new StoreScanner(scanInfo, ScanType.COMPACT_RETAIN_DELETES, scanners);
-    List<Cell> results = new ArrayList<>();
-    boolean hasMore = true;
-
-    while (hasMore) {
-      hasMore = scanner.next(results);
-      size += results.size();
-      results.clear();
-    }
-    scanner.close();
-    return size;
-  }
-
-  private static ExecutorService createThreadPool() {
-    int maxThreads = 10;
-    long keepAliveTime = 60;
-    final SynchronousQueue<Runnable> queue = new SynchronousQueue<>();
-    ThreadPoolExecutor pool = new ThreadPoolExecutor(1, maxThreads, keepAliveTime,
-      TimeUnit.SECONDS, queue, Threads.newDaemonThreadFactory("MobFileCompactionChore"),
-      new RejectedExecutionHandler() {
-        @Override
-        public void rejectedExecution(Runnable r, ThreadPoolExecutor executor) {
-          try {
-            // waiting for a thread to pick up instead of throwing exceptions.
-            queue.put(r);
-          } catch (InterruptedException e) {
-            throw new RejectedExecutionException(e);
-          }
-        }
-      });
-    ((ThreadPoolExecutor) pool).allowCoreThreadTimeOut(true);
-    return pool;
-  }
-
-  /**
-   * Resets the configuration.
-   */
-  private void resetConf() {
-    conf.setLong(MobConstants.MOB_COMPACTION_MERGEABLE_THRESHOLD,
-      MobConstants.DEFAULT_MOB_COMPACTION_MERGEABLE_THRESHOLD);
-    conf.setInt(MobConstants.MOB_DELFILE_MAX_COUNT, MobConstants.DEFAULT_MOB_DELFILE_MAX_COUNT);
-    conf.setInt(MobConstants.MOB_COMPACTION_BATCH_SIZE,
-      MobConstants.DEFAULT_MOB_COMPACTION_BATCH_SIZE);
-  }
-
-  /**
-   * The customized Distributed File System Implementation
-   */
-  static class FaultyDistributedFileSystem extends DistributedFileSystem {
-    private volatile boolean throwException = false;
-
-    public FaultyDistributedFileSystem() {
-      super();
-    }
-
-    public void setThrowException(boolean throwException) {
-      this.throwException = throwException;
-    }
-
-    @Override
-    public boolean rename(Path src, Path dst) throws IOException {
-      if (throwException) {
-        throw new IOException("No more files allowed");
-      }
-      return super.rename(src, dst);
-    }
-  }
-}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoadReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoadReplication.java
index 33ab4e3..30e456e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoadReplication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoadReplication.java
@@ -69,7 +69,7 @@ import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
 import org.apache.hadoop.hbase.mob.MobConstants;
 import org.apache.hadoop.hbase.mob.MobFileName;
 import org.apache.hadoop.hbase.mob.MobUtils;
-import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactor;
+//import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactor;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.TestReplicationBase;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -83,6 +83,7 @@ import org.junit.After;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
+import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -257,20 +258,21 @@ public class TestBulkLoadReplication extends TestReplicationBase {
     assertEquals(9, BULK_LOADS_COUNT.get());
   }
 
+  @Ignore
   @Test
   public void testPartionedMOBCompactionBulkLoadDoesntReplicate() throws Exception {
-    Path path = createMobFiles(UTIL3);
+//    Path path = createMobFiles(UTIL3);
     ColumnFamilyDescriptor descriptor =
       new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(famName);
     ExecutorService pool = null;
     try {
       pool = Executors.newFixedThreadPool(1);
-      PartitionedMobCompactor compactor =
-        new PartitionedMobCompactor(UTIL3.getConfiguration(), UTIL3.getTestFileSystem(), tableName,
-          descriptor, pool);
+//      PartitionedMobCompactor compactor =
+//        new PartitionedMobCompactor(UTIL3.getConfiguration(), UTIL3.getTestFileSystem(), tableName,
+//          descriptor, pool);
       BULK_LOAD_LATCH = new CountDownLatch(1);
       BULK_LOADS_COUNT.set(0);
-      compactor.compact(Arrays.asList(UTIL3.getTestFileSystem().listStatus(path)), true);
+//      compactor.compact(Arrays.asList(UTIL3.getTestFileSystem().listStatus(path)), true);
       assertTrue(BULK_LOAD_LATCH.await(1, TimeUnit.SECONDS));
       Thread.sleep(400);
       assertEquals(1, BULK_LOADS_COUNT.get());
@@ -348,35 +350,36 @@ public class TestBulkLoadReplication extends TestReplicationBase {
     return hFileLocation.getAbsoluteFile().getAbsolutePath();
   }
 
-  private Path createMobFiles(HBaseTestingUtility util) throws IOException {
-    Path testDir = FSUtils.getRootDir(util.getConfiguration());
-    Path mobTestDir = new Path(testDir, MobConstants.MOB_DIR_NAME);
-    Path basePath = new Path(new Path(mobTestDir, tableName.getNameAsString()), "f");
-    HFileContext meta = new HFileContextBuilder().withBlockSize(8 * 1024).build();
-    MobFileName mobFileName = null;
-    byte[] mobFileStartRow = new byte[32];
-    for (byte rowKey : Bytes.toBytes("01234")) {
-      mobFileName = MobFileName.create(mobFileStartRow, MobUtils.formatDate(new Date()),
-        UUID.randomUUID().toString().replaceAll("-", ""));
-      StoreFileWriter mobFileWriter =
-        new StoreFileWriter.Builder(util.getConfiguration(),
-          new CacheConfig(util.getConfiguration()), util.getTestFileSystem()).withFileContext(meta)
-          .withFilePath(new Path(basePath, mobFileName.getFileName())).build();
-      long now = System.currentTimeMillis();
-      try {
-        for (int i = 0; i < 10; i++) {
-          byte[] key = Bytes.add(Bytes.toBytes(rowKey), Bytes.toBytes(i));
-          byte[] dummyData = new byte[5000];
-          new Random().nextBytes(dummyData);
-          mobFileWriter.append(
-            new KeyValue(key, famName, Bytes.toBytes("1"), now, KeyValue.Type.Put, dummyData));
-        }
-      } finally {
-        mobFileWriter.close();
-      }
-    }
-    return basePath;
-  }
+
+//  private Path createMobFiles(HBaseTestingUtility util) throws IOException {
+//    Path testDir = FSUtils.getRootDir(util.getConfiguration());
+//    Path mobTestDir = new Path(testDir, MobConstants.MOB_DIR_NAME);
+//    Path basePath = new Path(new Path(mobTestDir, tableName.getNameAsString()), "f");
+//    HFileContext meta = new HFileContextBuilder().withBlockSize(8 * 1024).build();
+//    MobFileName mobFileName = null;
+//    byte[] mobFileStartRow = new byte[32];
+//    for (byte rowKey : Bytes.toBytes("01234")) {
+//      mobFileName = MobFileName.create(mobFileStartRow, MobUtils.formatDate(new Date()),
+//        UUID.randomUUID().toString().replaceAll("-", ""));
+//      StoreFileWriter mobFileWriter =
+//        new StoreFileWriter.Builder(util.getConfiguration(),
+//          new CacheConfig(util.getConfiguration()), util.getTestFileSystem()).withFileContext(meta)
+//          .withFilePath(new Path(basePath, mobFileName.getFileName())).build();
+//      long now = System.currentTimeMillis();
+//      try {
+//        for (int i = 0; i < 10; i++) {
+//          byte[] key = Bytes.add(Bytes.toBytes(rowKey), Bytes.toBytes(i));
+//          byte[] dummyData = new byte[5000];
+//          new Random().nextBytes(dummyData);
+//          mobFileWriter.append(
+//            new KeyValue(key, famName, Bytes.toBytes("1"), now, KeyValue.Type.Put, dummyData));
+//        }
+//      } finally {
+//        mobFileWriter.close();
+//      }
+//    }
+//    return basePath;
+//  }
 
   public static class BulkReplicationTestObserver implements RegionCoprocessor {
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreCompaction.java
index 74c8d79..fa44c18 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreCompaction.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreCompaction.java
@@ -270,30 +270,6 @@ public class TestMobStoreCompaction {
     // region.compactStores();
     region.compact(true);
     assertEquals("After compaction: store files", 1, countStoreFiles());
-    // still have original mob hfiles and now added a mob del file
-    assertEquals("After compaction: mob files", numHfiles + 1, countMobFiles());
-
-    Scan scan = new Scan();
-    scan.setRaw(true);
-    InternalScanner scanner = region.getScanner(scan);
-    List<Cell> results = new ArrayList<>();
-    scanner.next(results);
-    int deleteCount = 0;
-    while (!results.isEmpty()) {
-      for (Cell c : results) {
-        if (c.getTypeByte() == KeyValue.Type.DeleteFamily.getCode()) {
-          deleteCount++;
-          assertTrue(Bytes.equals(CellUtil.cloneRow(c), deleteRow));
-        }
-      }
-      results.clear();
-      scanner.next(results);
-    }
-    // assert the delete mark is retained after the major compaction
-    assertEquals(1, deleteCount);
-    scanner.close();
-    // assert the deleted cell is not counted
-    assertEquals("The cells in mob files", numHfiles - 1, countMobCellsInMobFiles(1));
   }
 
   private int countStoreFiles() throws IOException {
@@ -424,38 +400,4 @@ public class TestMobStoreCompaction {
     return files.size();
   }
 
-  private int countMobCellsInMobFiles(int expectedNumDelfiles) throws IOException {
-    Configuration copyOfConf = new Configuration(conf);
-    copyOfConf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0f);
-    CacheConfig cacheConfig = new CacheConfig(copyOfConf);
-    Path mobDirPath = MobUtils.getMobFamilyPath(conf, htd.getTableName(), hcd.getNameAsString());
-    List<HStoreFile> sfs = new ArrayList<>();
-    int numDelfiles = 0;
-    int size = 0;
-    if (fs.exists(mobDirPath)) {
-      for (FileStatus f : fs.listStatus(mobDirPath)) {
-        HStoreFile sf = new HStoreFile(fs, f.getPath(), conf, cacheConfig, BloomType.NONE, true);
-        sfs.add(sf);
-        if (StoreFileInfo.isDelFile(sf.getPath())) {
-          numDelfiles++;
-        }
-      }
-
-      List<StoreFileScanner> scanners = StoreFileScanner.getScannersForStoreFiles(sfs, false, true,
-        false, false, HConstants.LATEST_TIMESTAMP);
-      long timeToPurgeDeletes = Math.max(conf.getLong("hbase.hstore.time.to.purge.deletes", 0), 0);
-      long ttl = HStore.determineTTLFromFamily(hcd);
-      ScanInfo scanInfo = new ScanInfo(copyOfConf, hcd, ttl, timeToPurgeDeletes,
-        CellComparatorImpl.COMPARATOR);
-      StoreScanner scanner = new StoreScanner(scanInfo, ScanType.COMPACT_DROP_DELETES, scanners);
-      try {
-        size += UTIL.countRows(scanner);
-      } finally {
-        scanner.close();
-      }
-    }
-    // assert the number of the existing del files
-    assertEquals(expectedNumDelfiles, numDelfiles);
-    return size;
-  }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java
index b6227fc..8c83790 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java
@@ -411,7 +411,7 @@ public class BaseTestHBaseFsck {
     String startKey = mobFileName.getStartKey();
     String date = mobFileName.getDate();
     return MobFileName.create(startKey, date,
-                              TEST_UTIL.getRandomUUID().toString().replaceAll("-", ""))
+                              TEST_UTIL.getRandomUUID().toString().replaceAll("-", ""), "abcdef")
       .getFileName();
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckMOB.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckMOB.java
index 3a3becd..9870b73 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckMOB.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckMOB.java
@@ -98,6 +98,7 @@ public class TestHBaseFsckMOB extends BaseTestHBaseFsck {
   /**
    * This creates a table and then corrupts a mob file.  Hbck should quarantine the file.
    */
+  @SuppressWarnings("deprecation")
   @Test
   public void testQuarantineCorruptMobFile() throws Exception {
     TableName table = TableName.valueOf(name.getMethodName());